From a5797b49f686e66376ae1470aff7552da00b2740 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 25 Mar 2025 10:26:51 +0800 Subject: [PATCH 001/118] save --- dbms/src/Operators/CTE.cpp | 38 +++++++++++++++++++++++++ dbms/src/Operators/CTE.h | 34 +++++++++++++++++++++++ dbms/src/Operators/CTESinkOp.h | 51 ++++++++++++++++++++++++++++++++++ 3 files changed, 123 insertions(+) create mode 100644 dbms/src/Operators/CTE.cpp create mode 100644 dbms/src/Operators/CTE.h create mode 100644 dbms/src/Operators/CTESinkOp.h diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp new file mode 100644 index 00000000000..81e87ce6475 --- /dev/null +++ b/dbms/src/Operators/CTE.cpp @@ -0,0 +1,38 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + + +#include + +#include + +namespace DB +{ +Block CTE::tryGetBlockAt(size_t idx) +{ + std::shared_lock lock(this->rw_lock); + auto block_num = this->blocks.size(); // TODO maybe in the disk + if (block_num <= idx) + return Block(); + // TODO maybe fetch block from disk + return this->blocks[idx]; +} + +void CTE::pushBlock(const Block & block) +{ + // TODO track memory + std::unique_lock lock(this->rw_lock); + this->blocks.push_back(block); // TODO consider spill +} +} // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h new file mode 100644 index 00000000000..e07313f1603 --- /dev/null +++ b/dbms/src/Operators/CTE.h @@ -0,0 +1,34 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include + +#include + +namespace DB +{ +class CTE +{ +public: + Block tryGetBlockAt(size_t idx); + void pushBlock(const Block & block); +private: + std::shared_mutex rw_lock; + Blocks blocks; + // TODO spill +}; +} // namespace DB diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h new file mode 100644 index 00000000000..34dda7c5a72 --- /dev/null +++ b/dbms/src/Operators/CTESinkOp.h @@ -0,0 +1,51 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +namespace DB +{ +class CTESinkOp : public SinkOp +{ +public: +CTESinkOp( + PipelineExecutorContext & exec_context_, + const String & req_id) + : SinkOp(exec_context_, req_id) + {} + + String getName() const override { return "CTESinkOp"; } + + bool canHandleSelectiveBlock() const override { return true; } + +protected: + void operatePrefixImpl() override; + void operateSuffixImpl() override; + + OperatorStatus writeImpl(Block && block) override; + + OperatorStatus prepareImpl() override; + + OperatorStatus awaitImpl() override; + +private: + OperatorStatus waitForWriter() const; + +private: + size_t total_rows = 0; + bool input_done = false; +}; +} // namespace DB From 385b051cdb3fbe17f314a33fe034bfe7e9456e99 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 25 Mar 2025 18:26:23 +0800 Subject: [PATCH 002/118] save --- dbms/src/Operators/CTE.cpp | 16 ++++++++++++---- dbms/src/Operators/CTE.h | 6 +++++- dbms/src/Operators/CTESink.Op.cpp | 29 +++++++++++++++++++++++++++++ dbms/src/Operators/CTESinkOp.h | 25 +++++++++++-------------- 4 files changed, 57 insertions(+), 19 deletions(-) create mode 100644 dbms/src/Operators/CTESink.Op.cpp diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 81e87ce6475..a6d4cff38d1 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -15,18 +15,20 @@ #include +#include #include +#include namespace DB { -Block CTE::tryGetBlockAt(size_t idx) +std::pair CTE::tryGetBlockAt(size_t idx) { std::shared_lock lock(this->rw_lock); - auto block_num = this->blocks.size(); // TODO maybe in the disk + auto block_num = this->blocks.size(); // TODO maybe blocks are in disk if (block_num <= idx) - return Block(); + return {this->is_eof, Block()}; // TODO maybe fetch block from disk - return this->blocks[idx]; + return {false, this->blocks[idx]}; } void CTE::pushBlock(const Block & block) @@ -35,4 +37,10 @@ void CTE::pushBlock(const Block & block) std::unique_lock lock(this->rw_lock); this->blocks.push_back(block); // TODO consider spill } + +void CTE::notifyEOF() +{ + std::unique_lock lock(this->rw_lock); + this->is_eof = true; +} } // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index e07313f1603..9363000dd70 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -18,17 +18,21 @@ #include #include +#include namespace DB { class CTE { public: - Block tryGetBlockAt(size_t idx); + std::pair tryGetBlockAt(size_t idx); void pushBlock(const Block & block); + void notifyEOF(); private: std::shared_mutex rw_lock; Blocks blocks; + + bool is_eof = false; // TODO spill }; } // namespace DB diff --git a/dbms/src/Operators/CTESink.Op.cpp b/dbms/src/Operators/CTESink.Op.cpp new file mode 100644 index 00000000000..0bbd9001c1d --- /dev/null +++ b/dbms/src/Operators/CTESink.Op.cpp @@ -0,0 +1,29 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include + +namespace DB +{ +void CTESinkOp::operateSuffixImpl() +{ + LOG_DEBUG(log, "finish write with {} rows", this->total_rows); +} + +OperatorStatus CTESinkOp::writeImpl(Block && block) +{ + this->cte->pushBlock(block); + return +} +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 34dda7c5a72..5af728f1f72 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -15,37 +15,34 @@ #pragma once #include +#include namespace DB { +// TODO handle selective block class CTESinkOp : public SinkOp { public: -CTESinkOp( - PipelineExecutorContext & exec_context_, - const String & req_id) - : SinkOp(exec_context_, req_id) - {} + CTESinkOp( + PipelineExecutorContext & exec_context_, + const String & req_id) + : SinkOp(exec_context_, req_id) + {} String getName() const override { return "CTESinkOp"; } - bool canHandleSelectiveBlock() const override { return true; } protected: - void operatePrefixImpl() override; void operateSuffixImpl() override; - OperatorStatus writeImpl(Block && block) override; - - OperatorStatus prepareImpl() override; - OperatorStatus awaitImpl() override; private: - OperatorStatus waitForWriter() const; - -private: + std::shared_ptr cte; size_t total_rows = 0; bool input_done = false; + + // When spill is triggered in CTE, we need to temporarily save the block + Block tmp_block; }; } // namespace DB From a8d3052d805f8ff422631b51ce9a046fcad5eac4 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 28 Mar 2025 19:11:46 +0800 Subject: [PATCH 003/118] save --- dbms/src/Operators/CTE.cpp | 40 ++++++++++++++++++-- dbms/src/Operators/CTE.h | 27 +++++++++++++- dbms/src/Operators/CTESink.Op.cpp | 5 ++- dbms/src/Operators/CTESinkOp.h | 5 ++- dbms/src/Operators/CTESource.cpp | 61 +++++++++++++++++++++++++++++++ dbms/src/Operators/CTESource.h | 58 +++++++++++++++++++++++++++++ 6 files changed, 187 insertions(+), 9 deletions(-) create mode 100644 dbms/src/Operators/CTESource.cpp create mode 100644 dbms/src/Operators/CTESource.h diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index a6d4cff38d1..1ba0f00fe5d 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -21,14 +21,35 @@ namespace DB { -std::pair CTE::tryGetBlockAt(size_t idx) +FetchStatus CTE::checkAvailableBlockAt(size_t idx) +{ + std::shared_lock lock(this->rw_lock); + auto block_num = this->blocks.size(); // TODO consider spill + if (block_num <= idx) + { + if (this->is_eof) + return FetchStatus::Eof; + else + return FetchStatus::Waiting; + } + // TODO handle FetchStatus::Cancelled + return FetchStatus::Ok; +} + +std::pair CTE::tryGetBlockAt(size_t idx) { std::shared_lock lock(this->rw_lock); auto block_num = this->blocks.size(); // TODO maybe blocks are in disk if (block_num <= idx) - return {this->is_eof, Block()}; + { + if (this->is_eof) + return {FetchStatus::Eof, Block()}; + else + return {FetchStatus::Waiting, Block()}; + } + // TODO handle error and cancel // TODO maybe fetch block from disk - return {false, this->blocks[idx]}; + return {FetchStatus::Ok, this->blocks[idx]}; } void CTE::pushBlock(const Block & block) @@ -43,4 +64,17 @@ void CTE::notifyEOF() std::unique_lock lock(this->rw_lock); this->is_eof = true; } + +void CTE::registerTask(TaskPtr && task) +{ + { + std::unique_lock lock(this->rw_lock); + if (!this->hasDataNoLock()) + { + pipe_cv.registerTask(std::move(task)); + return; + } + } + this->pipe_cv.notifyTaskDirectly(std::move(task)); +} } // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 9363000dd70..861f54688e3 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -16,23 +16,46 @@ #include #include +#include +#include #include #include namespace DB { -class CTE +enum class FetchStatus +{ + Ok, + Waiting, + Eof, + Cancelled +}; + +class CTE : public NotifyFuture { public: - std::pair tryGetBlockAt(size_t idx); + std::pair tryGetBlockAt(size_t idx); + FetchStatus checkAvailableBlockAt(size_t idx); void pushBlock(const Block & block); void notifyEOF(); + + void registerTask(TaskPtr && task) override; + private: + // Return true if CTE has data + inline bool hasDataNoLock() const { return !this->blocks.empty() || this->spill_triggered; } + std::shared_mutex rw_lock; Blocks blocks; + // Tasks in WAITING_FOR_NOTIFY are saved in this deque + std::deque waiting_tasks; + PipeConditionVariable pipe_cv; + bool is_eof = false; + + bool spill_triggered = false; // TODO this var may be useless, just a placement so far // TODO spill }; } // namespace DB diff --git a/dbms/src/Operators/CTESink.Op.cpp b/dbms/src/Operators/CTESink.Op.cpp index 0bbd9001c1d..e90a8e5e8bf 100644 --- a/dbms/src/Operators/CTESink.Op.cpp +++ b/dbms/src/Operators/CTESink.Op.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include "Operators/Operator.h" namespace DB { @@ -23,7 +24,7 @@ void CTESinkOp::operateSuffixImpl() OperatorStatus CTESinkOp::writeImpl(Block && block) { - this->cte->pushBlock(block); - return + this->cte->pushBlock(block); // TODO handle spill + return OperatorStatus::NEED_INPUT; } } // namespace DB \ No newline at end of file diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 5af728f1f72..7724c6b4301 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -25,8 +25,9 @@ class CTESinkOp : public SinkOp public: CTESinkOp( PipelineExecutorContext & exec_context_, - const String & req_id) - : SinkOp(exec_context_, req_id) + const String & req_id, + std::shared_ptr cte_) + : SinkOp(exec_context_, req_id), cte(cte_) {} String getName() const override { return "CTESinkOp"; } diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp new file mode 100644 index 00000000000..5e68655d6e5 --- /dev/null +++ b/dbms/src/Operators/CTESource.cpp @@ -0,0 +1,61 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include "Operators/CTE.h" +#include "Operators/Operator.h" + +namespace DB +{ +// TODO in some cases, source needs to manually filter some data when cte saves all data +OperatorStatus CTESourceOp::readImpl(Block & block) +{ + // TODO when spill is triggered, this function may be blocked by spill. fix it + auto res = this->cte->tryGetBlockAt(this->block_fetch_idx); + switch (res.first) + { + case DB::FetchStatus::Eof: + case DB::FetchStatus::Ok: + block = res.second; + ++(this->block_fetch_idx); + return OperatorStatus::HAS_OUTPUT; + case DB::FetchStatus::Waiting: + if unlikely (this->block_fetch_idx == 0) + // CTE has not begun to receive data yet when block_fetch_idx == 0 + // So we need to wait the notify from CTE + return OperatorStatus::WAIT_FOR_NOTIFY; + else + return OperatorStatus::WAITING; + case DB::FetchStatus::Cancelled: + return OperatorStatus::CANCELLED; + } +} + +OperatorStatus CTESourceOp::awaitImpl() +{ + // TODO when spill is triggered, this function may be blocked by spill. fix it + auto res = this->cte->checkAvailableBlockAt(this->block_fetch_idx); + switch (res) + { + case DB::FetchStatus::Eof: + case DB::FetchStatus::Ok: + // Do not add block_fetch_idx here, as we just judge if there are available blocks + return OperatorStatus::HAS_OUTPUT; + case DB::FetchStatus::Waiting: + return OperatorStatus::WAITING; + case DB::FetchStatus::Cancelled: + return OperatorStatus::CANCELLED; + } +} +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h new file mode 100644 index 00000000000..1d0524cfe43 --- /dev/null +++ b/dbms/src/Operators/CTESource.h @@ -0,0 +1,58 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ +// TODO consider fine granine +// TODO maybe consider handling selective block +class CTESourceOp : public SourceOp +{ +public: + CTESourceOp( + PipelineExecutorContext & exec_context_, + const String & req_id, + std::shared_ptr cte_) + : SourceOp(exec_context_, req_id), cte(cte_) {} + + String getName() const override { return "CTESourceOp"; } + + IOProfileInfoPtr getIOProfileInfo() const override { return io_profile_info; } + +protected: + void operateSuffixImpl() override { LOG_DEBUG(log, "finish read {} rows from cte source", total_rows); } + + OperatorStatus readImpl(Block & block) override; + + OperatorStatus awaitImpl() override; + +private: + std::shared_ptr cte; + size_t block_fetch_idx = 0; + + uint64_t total_rows{}; + std::queue block_queue; + + // size_t stream_id; // TODO maybe need it + + IOProfileInfoPtr io_profile_info; // TODO initialize it and handle it +}; +} // namespace DB From 97f44e0a8d72fd3c76e74c9241a3bb34acbfa2ed Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 31 Mar 2025 19:30:43 +0800 Subject: [PATCH 004/118] save --- dbms/src/Flash/Mpp/CTEManager.cpp | 47 +++++++++++++++++++++++++++++++ dbms/src/Flash/Mpp/CTEManager.h | 34 ++++++++++++++++++++++ dbms/src/Operators/CTE.h | 3 ++ dbms/src/Operators/CTESinkOp.h | 1 - 4 files changed, 84 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Flash/Mpp/CTEManager.cpp create mode 100644 dbms/src/Flash/Mpp/CTEManager.h diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp new file mode 100644 index 00000000000..a681e83368f --- /dev/null +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -0,0 +1,47 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +namespace DB +{ +std::shared_ptr CTEManager::getCTE(const String & query_id_and_cte_id) +{ + std::lock_guard lock(this->mu); + auto iter = this->ctes.find(query_id_and_cte_id); + if (iter == this->ctes.end()) + { + // It's the first time we request for the specific cte + // Create it because no one created it before. + auto cte = std::make_shared(); + this->ctes[query_id_and_cte_id] = std::make_pair(1, cte); + return cte; + } + + ++(iter->second.first); + return iter->second.second; +} + +void CTEManager::releaseCTE(const String & query_id_and_cte_id) +{ + std::lock_guard lock(this->mu); + auto iter = this->ctes.find(query_id_and_cte_id); + if unlikely (iter == this->ctes.end()) + throw Exception(fmt::format("Can't find cte: {}", query_id_and_cte_id)); + --(iter->second.first); + if (iter->second.first == 0) + this->ctes.erase(iter); +} +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Mpp/CTEManager.h b/dbms/src/Flash/Mpp/CTEManager.h new file mode 100644 index 00000000000..25ea9617bfe --- /dev/null +++ b/dbms/src/Flash/Mpp/CTEManager.h @@ -0,0 +1,34 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +#include +#include +#include + +namespace DB +{ +class CTEManager +{ +public: + std::shared_ptr getCTE(const String & query_id_and_cte_id); + void releaseCTE(const String & query_id_and_cte_id); +private: + std::mutex mu; + std::map>> ctes; +}; +} // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 861f54688e3..a073d153495 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -32,9 +32,12 @@ enum class FetchStatus Cancelled }; +// TODO track memory with allocator class CTE : public NotifyFuture { public: + ~CTE() override = default; + std::pair tryGetBlockAt(size_t idx); FetchStatus checkAvailableBlockAt(size_t idx); void pushBlock(const Block & block); diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 7724c6b4301..533b8976158 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -36,7 +36,6 @@ class CTESinkOp : public SinkOp protected: void operateSuffixImpl() override; OperatorStatus writeImpl(Block && block) override; - OperatorStatus awaitImpl() override; private: std::shared_ptr cte; From 00580c498c4bb6b6a81c8c780bdb56964410e518 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 1 Apr 2025 12:24:12 +0800 Subject: [PATCH 005/118] save --- dbms/src/Operators/CTE.cpp | 12 ++++++++++-- dbms/src/Operators/CTE.h | 3 ++- dbms/src/Operators/CTESink.Op.cpp | 7 +++++++ dbms/src/Operators/CTESinkOp.h | 4 ---- dbms/src/Operators/CTESource.h | 3 +-- 5 files changed, 20 insertions(+), 9 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 1ba0f00fe5d..0b697797a44 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -54,15 +54,23 @@ std::pair CTE::tryGetBlockAt(size_t idx) void CTE::pushBlock(const Block & block) { - // TODO track memory std::unique_lock lock(this->rw_lock); - this->blocks.push_back(block); // TODO consider spill + + // TODO consider spill + this->memory_usage += block.bytes(); + // TODO check spill + if unlikely (this->blocks.empty()) + this->pipe_cv.notifyAll(); + this->blocks.push_back(block); } void CTE::notifyEOF() { std::unique_lock lock(this->rw_lock); this->is_eof = true; + + // Just in case someone is in WAITING_FOR_NOTIFY status + this->pipe_cv.notifyAll(); } void CTE::registerTask(TaskPtr && task) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index a073d153495..9efdba2ee25 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -52,12 +52,13 @@ class CTE : public NotifyFuture std::shared_mutex rw_lock; Blocks blocks; + size_t memory_usage = 0; + // Tasks in WAITING_FOR_NOTIFY are saved in this deque std::deque waiting_tasks; PipeConditionVariable pipe_cv; bool is_eof = false; - bool spill_triggered = false; // TODO this var may be useless, just a placement so far // TODO spill }; diff --git a/dbms/src/Operators/CTESink.Op.cpp b/dbms/src/Operators/CTESink.Op.cpp index e90a8e5e8bf..23f2eee52fd 100644 --- a/dbms/src/Operators/CTESink.Op.cpp +++ b/dbms/src/Operators/CTESink.Op.cpp @@ -24,6 +24,13 @@ void CTESinkOp::operateSuffixImpl() OperatorStatus CTESinkOp::writeImpl(Block && block) { + if (!block) + { + this->input_done = true; + this->cte->notifyEOF(); + return OperatorStatus::FINISHED; + } + this->total_rows += block.rows(); this->cte->pushBlock(block); // TODO handle spill return OperatorStatus::NEED_INPUT; } diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 533b8976158..f08571d5241 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -19,7 +19,6 @@ namespace DB { -// TODO handle selective block class CTESinkOp : public SinkOp { public: @@ -41,8 +40,5 @@ class CTESinkOp : public SinkOp std::shared_ptr cte; size_t total_rows = 0; bool input_done = false; - - // When spill is triggered in CTE, we need to temporarily save the block - Block tmp_block; }; } // namespace DB diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 1d0524cfe43..d22efb44c79 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -22,8 +22,7 @@ namespace DB { -// TODO consider fine granine -// TODO maybe consider handling selective block +// TODO consider fine graine (start from here) class CTESourceOp : public SourceOp { public: From 966fa7148bf4c411c4dc11dc82a7d8a55ce172d6 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 2 Apr 2025 15:00:10 +0800 Subject: [PATCH 006/118] save --- dbms/src/Flash/Planner/PhysicalPlanNode.h | 5 +- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 0 .../src/Flash/Planner/Plans/PhysicalCTESink.h | 0 .../Flash/Planner/Plans/PhysicalCTESource.cpp | 29 ++++++++++++ .../Flash/Planner/Plans/PhysicalCTESource.h | 47 +++++++++++++++++++ dbms/src/Operators/CTESource.h | 8 +--- 6 files changed, 81 insertions(+), 8 deletions(-) create mode 100644 dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp create mode 100644 dbms/src/Flash/Planner/Plans/PhysicalCTESink.h create mode 100644 dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp create mode 100644 dbms/src/Flash/Planner/Plans/PhysicalCTESource.h diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.h b/dbms/src/Flash/Planner/PhysicalPlanNode.h index ea33264eef0..8d6ce3749f2 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.h +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.h @@ -102,7 +102,10 @@ class PhysicalPlanNode : public std::enable_shared_from_this /// Used for non-fine grained shuffle sink plan node to trigger two-stage execution logic. virtual EventPtr doSinkComplete(PipelineExecutorContext & /*exec_status*/); - virtual void buildBlockInputStreamImpl(DAGPipeline & /*pipeline*/, Context & /*context*/, size_t /*max_streams*/){}; + virtual void buildBlockInputStreamImpl(DAGPipeline & /*pipeline*/, Context & /*context*/, size_t /*max_streams*/) + { + throw Exception("Unsupported"); + }; virtual void buildPipelineExecGroupImpl( PipelineExecutorContext & /*exec_status*/, diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp new file mode 100644 index 00000000000..a73d0a885cc --- /dev/null +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -0,0 +1,29 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +namespace DB +{ +void PhysicalCTESource::finalizeImpl(const Names & parent_require) +{ + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); +} + +const Block & PhysicalCTESource::getSampleBlock() const +{ + return sample_block; +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h new file mode 100644 index 00000000000..d3d3821811b --- /dev/null +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -0,0 +1,47 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include + +namespace DB +{ +class PhysicalCTESource : public PhysicalLeaf +{ +public: + static PhysicalPlanNodePtr build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const FineGrainedShuffle & fine_grained_shuffle); + + // TODO + void buildPipelineExecGroupImpl( + PipelineExecutorContext & exec_context, + PipelineExecGroupBuilder & group_builder, + Context & context, + size_t concurrency) override; + + void finalizeImpl(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + Block sample_block; + String query_id_and_cte_id; + std::shared_ptr cte; +}; +} // namespace DB diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index d22efb44c79..da0e62a0073 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -22,7 +22,7 @@ namespace DB { -// TODO consider fine graine (start from here) +// TODO consider fine graine at compiling stage class CTESourceOp : public SourceOp { public: @@ -34,8 +34,6 @@ class CTESourceOp : public SourceOp String getName() const override { return "CTESourceOp"; } - IOProfileInfoPtr getIOProfileInfo() const override { return io_profile_info; } - protected: void operateSuffixImpl() override { LOG_DEBUG(log, "finish read {} rows from cte source", total_rows); } @@ -49,9 +47,5 @@ class CTESourceOp : public SourceOp uint64_t total_rows{}; std::queue block_queue; - - // size_t stream_id; // TODO maybe need it - - IOProfileInfoPtr io_profile_info; // TODO initialize it and handle it }; } // namespace DB From c2fb2332fb537f892a35ee856b9b277e4b6491ae Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 3 Apr 2025 09:28:39 +0800 Subject: [PATCH 007/118] save --- dbms/src/Flash/Planner/PlanType.h | 2 + .../Flash/Planner/Plans/PhysicalCTESink.cpp | 77 +++++++++++++++++++ .../src/Flash/Planner/Plans/PhysicalCTESink.h | 54 +++++++++++++ .../Flash/Planner/Plans/PhysicalCTESource.cpp | 20 +++++ .../Flash/Planner/Plans/PhysicalCTESource.h | 24 ++++-- 5 files changed, 171 insertions(+), 6 deletions(-) diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index 37422ab8fb0..26010abdd8b 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -42,6 +42,8 @@ struct PlanType JoinBuild = 17, JoinProbe = 18, GetResult = 19, + CTESource = 20, + CTESink = 21, }; PlanTypeEnum enum_value; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index e69de29bb2d..fe29793d1af 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -0,0 +1,77 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include + +#include + +namespace DB +{ +PhysicalPlanNodePtr PhysicalCTESink::build( + const String & executor_id, + const LoggerPtr & log, + const tipb::ExchangeSender & exchange_sender, + const FineGrainedShuffle & fine_grained_shuffle, + const PhysicalPlanNodePtr & child) +{ + RUNTIME_CHECK(child); + + // std::vector partition_col_ids = ExchangeSenderInterpreterHelper::genPartitionColIds(exchange_sender); + // TiDB::TiDBCollators partition_col_collators + // = ExchangeSenderInterpreterHelper::genPartitionColCollators(exchange_sender); + + // auto physical_exchange_sender = std::make_shared( + // executor_id, + // child->getSchema(), + // fine_grained_shuffle, + // log->identifier(), + // child, + // partition_col_ids, + // partition_col_collators, + // exchange_sender.tp(), + // exchange_sender.compression()); + // // executeUnion will be call after sender.transform, so don't need to restore concurrency. + // physical_exchange_sender->disableRestoreConcurrency(); + // return physical_exchange_sender; +} + +void PhysicalCTESink::buildPipelineExecGroupImpl( + PipelineExecutorContext & exec_context, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) +{ + size_t partition_id = 0; + group_builder.transform([&](auto & builder) { + std::shared_ptr cte; // TODO get it from CTEManager + builder.setSinkOp( + std::make_unique(exec_context, log->identifier(), cte)); + ++partition_id; + }); +} + +void PhysicalCTESink::finalizeImpl(const Names & parent_require) +{ + child->finalize(parent_require); +} + +const Block & PhysicalCTESink::getSampleBlock() const +{ + return child->getSampleBlock(); +} + +} // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h index e69de29bb2d..7cb0164d1d6 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h @@ -0,0 +1,54 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +class PhysicalCTESink : public PhysicalUnary +{ +public: + static PhysicalPlanNodePtr build( + const String & executor_id, + const LoggerPtr & log, + const tipb::ExchangeSender & exchange_sender, + const FineGrainedShuffle & fine_grained_shuffle, + const PhysicalPlanNodePtr & child); + + PhysicalCTESink( + const String & executor_id_, + const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, + const String & req_id, + const PhysicalPlanNodePtr & child_) + : PhysicalUnary(executor_id_, PlanType::CTESink, schema_, fine_grained_shuffle_, req_id, child_) + {} + + void finalizeImpl(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void buildPipelineExecGroupImpl( + PipelineExecutorContext & exec_context, + PipelineExecGroupBuilder & group_builder, + Context & context, + size_t /*concurrency*/) override; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index a73d0a885cc..38d83f0fd6c 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -17,6 +17,26 @@ namespace DB { +void PhysicalCTESource::buildPipelineExecGroupImpl( + PipelineExecutorContext & exec_context, + PipelineExecGroupBuilder & group_builder, + Context & context, + size_t concurrency) +{ + if (fine_grained_shuffle.enabled()) + concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); + + for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) + { + // group_builder.addConcurrency(std::make_unique( + // exec_context, + // log->identifier(), + // mpp_exchange_receiver, + // /*stream_id=*/fine_grained_shuffle.enabled() ? partition_id : 0)); + } + // context.getDAGContext()->addInboundIOProfileInfos(executor_id, group_builder.getCurIOProfileInfos()); +} + void PhysicalCTESource::finalizeImpl(const Names & parent_require) { FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h index d3d3821811b..acda6d2177c 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -28,17 +28,29 @@ class PhysicalCTESource : public PhysicalLeaf const LoggerPtr & log, const FineGrainedShuffle & fine_grained_shuffle); - // TODO - void buildPipelineExecGroupImpl( - PipelineExecutorContext & exec_context, - PipelineExecGroupBuilder & group_builder, - Context & context, - size_t concurrency) override; + // TODO to partition data, we may need to call `ExchangeSenderInterpreterHelper::genPartitionColCollators` like PhysicalExchangeSender + PhysicalCTESource( + const String & executor_id_, + const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle, + const String & req_id, + const Block & sample_block_) + : PhysicalLeaf(executor_id_, PlanType::CTESource, schema_, fine_grained_shuffle, req_id) + , sample_block(sample_block_) + {} void finalizeImpl(const Names & parent_require) override; const Block & getSampleBlock() const override; +private: + // TODO + void buildPipelineExecGroupImpl( + PipelineExecutorContext & exec_context, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) override; + private: Block sample_block; String query_id_and_cte_id; From 56df905933143ddc4b000c376766c88c202d24ac Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 3 Apr 2025 09:53:08 +0800 Subject: [PATCH 008/118] format --- dbms/src/Flash/Mpp/CTEManager.cpp | 2 +- dbms/src/Flash/Mpp/CTEManager.h | 1 + dbms/src/Flash/Planner/PhysicalPlanNode.h | 2 +- dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp | 5 ++--- dbms/src/Flash/Planner/Plans/PhysicalCTESink.h | 12 ++++++------ .../Flash/Planner/Plans/PhysicalCTESource.cpp | 2 +- dbms/src/Flash/Planner/Plans/PhysicalCTESource.h | 16 ++++++++-------- dbms/src/Operators/CTE.cpp | 4 ++-- dbms/src/Operators/CTE.h | 2 +- dbms/src/Operators/CTESink.Op.cpp | 3 ++- dbms/src/Operators/CTESinkOp.h | 12 +++++------- dbms/src/Operators/CTESource.cpp | 5 +++-- dbms/src/Operators/CTESource.h | 11 +++++------ 13 files changed, 38 insertions(+), 39 deletions(-) diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index a681e83368f..f2d054c0ab9 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -44,4 +44,4 @@ void CTEManager::releaseCTE(const String & query_id_and_cte_id) if (iter->second.first == 0) this->ctes.erase(iter); } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Mpp/CTEManager.h b/dbms/src/Flash/Mpp/CTEManager.h index 25ea9617bfe..d062bee0b53 100644 --- a/dbms/src/Flash/Mpp/CTEManager.h +++ b/dbms/src/Flash/Mpp/CTEManager.h @@ -27,6 +27,7 @@ class CTEManager public: std::shared_ptr getCTE(const String & query_id_and_cte_id); void releaseCTE(const String & query_id_and_cte_id); + private: std::mutex mu; std::map>> ctes; diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.h b/dbms/src/Flash/Planner/PhysicalPlanNode.h index 8d6ce3749f2..19f3f68cf9b 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.h +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.h @@ -105,7 +105,7 @@ class PhysicalPlanNode : public std::enable_shared_from_this virtual void buildBlockInputStreamImpl(DAGPipeline & /*pipeline*/, Context & /*context*/, size_t /*max_streams*/) { throw Exception("Unsupported"); - }; + } virtual void buildPipelineExecGroupImpl( PipelineExecutorContext & /*exec_status*/, diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index fe29793d1af..a3760a83be4 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include +#include #include #include @@ -58,8 +58,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( size_t partition_id = 0; group_builder.transform([&](auto & builder) { std::shared_ptr cte; // TODO get it from CTEManager - builder.setSinkOp( - std::make_unique(exec_context, log->identifier(), cte)); + builder.setSinkOp(std::make_unique(exec_context, log->identifier(), cte)); ++partition_id; }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h index 7cb0164d1d6..06c92327043 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h @@ -31,12 +31,12 @@ class PhysicalCTESink : public PhysicalUnary const FineGrainedShuffle & fine_grained_shuffle, const PhysicalPlanNodePtr & child); - PhysicalCTESink( - const String & executor_id_, - const NamesAndTypes & schema_, - const FineGrainedShuffle & fine_grained_shuffle_, - const String & req_id, - const PhysicalPlanNodePtr & child_) + PhysicalCTESink( + const String & executor_id_, + const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, + const String & req_id, + const PhysicalPlanNodePtr & child_) : PhysicalUnary(executor_id_, PlanType::CTESink, schema_, fine_grained_shuffle_, req_id, child_) {} diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 38d83f0fd6c..7debf3b6e0a 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include +#include namespace DB { diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h index acda6d2177c..3cc83d04092 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -30,14 +30,14 @@ class PhysicalCTESource : public PhysicalLeaf // TODO to partition data, we may need to call `ExchangeSenderInterpreterHelper::genPartitionColCollators` like PhysicalExchangeSender PhysicalCTESource( - const String & executor_id_, - const NamesAndTypes & schema_, - const FineGrainedShuffle & fine_grained_shuffle, - const String & req_id, - const Block & sample_block_) - : PhysicalLeaf(executor_id_, PlanType::CTESource, schema_, fine_grained_shuffle, req_id) - , sample_block(sample_block_) - {} + const String & executor_id_, + const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle, + const String & req_id, + const Block & sample_block_) + : PhysicalLeaf(executor_id_, PlanType::CTESource, schema_, fine_grained_shuffle, req_id) + , sample_block(sample_block_) + {} void finalizeImpl(const Names & parent_require) override; diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 0b697797a44..8e0584ee32f 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -35,7 +35,7 @@ FetchStatus CTE::checkAvailableBlockAt(size_t idx) // TODO handle FetchStatus::Cancelled return FetchStatus::Ok; } - + std::pair CTE::tryGetBlockAt(size_t idx) { std::shared_lock lock(this->rw_lock); @@ -55,7 +55,7 @@ std::pair CTE::tryGetBlockAt(size_t idx) void CTE::pushBlock(const Block & block) { std::unique_lock lock(this->rw_lock); - + // TODO consider spill this->memory_usage += block.bytes(); // TODO check spill diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 9efdba2ee25..13adcc998f1 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -14,8 +14,8 @@ #pragma once -#include #include +#include #include #include diff --git a/dbms/src/Operators/CTESink.Op.cpp b/dbms/src/Operators/CTESink.Op.cpp index 23f2eee52fd..044b215b341 100644 --- a/dbms/src/Operators/CTESink.Op.cpp +++ b/dbms/src/Operators/CTESink.Op.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include + #include "Operators/Operator.h" namespace DB @@ -34,4 +35,4 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) this->cte->pushBlock(block); // TODO handle spill return OperatorStatus::NEED_INPUT; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index f08571d5241..a69f552a15f 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -14,20 +14,18 @@ #pragma once -#include #include +#include namespace DB { class CTESinkOp : public SinkOp { public: - CTESinkOp( - PipelineExecutorContext & exec_context_, - const String & req_id, - std::shared_ptr cte_) - : SinkOp(exec_context_, req_id), cte(cte_) - {} + CTESinkOp(PipelineExecutorContext & exec_context_, const String & req_id, std::shared_ptr cte_) + : SinkOp(exec_context_, req_id) + , cte(cte_) + {} String getName() const override { return "CTESinkOp"; } bool canHandleSelectiveBlock() const override { return true; } diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index 5e68655d6e5..1e6e9df5ae5 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include + #include "Operators/CTE.h" #include "Operators/Operator.h" @@ -41,7 +42,7 @@ OperatorStatus CTESourceOp::readImpl(Block & block) return OperatorStatus::CANCELLED; } } - + OperatorStatus CTESourceOp::awaitImpl() { // TODO when spill is triggered, this function may be blocked by spill. fix it @@ -58,4 +59,4 @@ OperatorStatus CTESourceOp::awaitImpl() return OperatorStatus::CANCELLED; } } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index da0e62a0073..2d7c212bc0c 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -17,8 +17,8 @@ #include #include #include -#include #include +#include namespace DB { @@ -26,11 +26,10 @@ namespace DB class CTESourceOp : public SourceOp { public: - CTESourceOp( - PipelineExecutorContext & exec_context_, - const String & req_id, - std::shared_ptr cte_) - : SourceOp(exec_context_, req_id), cte(cte_) {} + CTESourceOp(PipelineExecutorContext & exec_context_, const String & req_id, std::shared_ptr cte_) + : SourceOp(exec_context_, req_id) + , cte(cte_) + {} String getName() const override { return "CTESourceOp"; } From 265da87d8c60c45b1b5c59ab1728c80b21212afb Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 3 Apr 2025 11:16:25 +0800 Subject: [PATCH 009/118] save --- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 29 ++++++--------- .../src/Flash/Planner/Plans/PhysicalCTESink.h | 1 - .../Flash/Planner/Plans/PhysicalCTESource.cpp | 36 +++++++++++++++---- 3 files changed, 40 insertions(+), 26 deletions(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index a3760a83be4..b62130932cd 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -24,29 +24,20 @@ namespace DB PhysicalPlanNodePtr PhysicalCTESink::build( const String & executor_id, const LoggerPtr & log, - const tipb::ExchangeSender & exchange_sender, const FineGrainedShuffle & fine_grained_shuffle, const PhysicalPlanNodePtr & child) { + // TODO CTEManager: get cte from CTEManager RUNTIME_CHECK(child); - // std::vector partition_col_ids = ExchangeSenderInterpreterHelper::genPartitionColIds(exchange_sender); - // TiDB::TiDBCollators partition_col_collators - // = ExchangeSenderInterpreterHelper::genPartitionColCollators(exchange_sender); - - // auto physical_exchange_sender = std::make_shared( - // executor_id, - // child->getSchema(), - // fine_grained_shuffle, - // log->identifier(), - // child, - // partition_col_ids, - // partition_col_collators, - // exchange_sender.tp(), - // exchange_sender.compression()); - // // executeUnion will be call after sender.transform, so don't need to restore concurrency. - // physical_exchange_sender->disableRestoreConcurrency(); - // return physical_exchange_sender; + auto physical_cte_sink = std::make_shared( + executor_id, + child->getSchema(), + fine_grained_shuffle, + log->identifier(), + child); + physical_cte_sink->disableRestoreConcurrency(); + return physical_cte_sink; } void PhysicalCTESink::buildPipelineExecGroupImpl( @@ -57,7 +48,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( { size_t partition_id = 0; group_builder.transform([&](auto & builder) { - std::shared_ptr cte; // TODO get it from CTEManager + std::shared_ptr cte; // TODO CTEManager: get it from CTEManager builder.setSinkOp(std::make_unique(exec_context, log->identifier(), cte)); ++partition_id; }); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h index 06c92327043..75376aafe95 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h @@ -27,7 +27,6 @@ class PhysicalCTESink : public PhysicalUnary static PhysicalPlanNodePtr build( const String & executor_id, const LoggerPtr & log, - const tipb::ExchangeSender & exchange_sender, const FineGrainedShuffle & fine_grained_shuffle, const PhysicalPlanNodePtr & child); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 7debf3b6e0a..ba4d1db3ec0 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -12,11 +12,37 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include #include #include +#include + +#include "Operators/CTESource.h" namespace DB { +PhysicalPlanNodePtr PhysicalCTESource::build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const FineGrainedShuffle & fine_grained_shuffle + /* TODO tipb::ExchangeReceiver */) +{ + // TODO tipb for cte: need output schema field in tipb for cte source + // TODO CTEManager: get cte from CTEManager + // TODO we need to get meta data such as `partition_col_collators` for partitioning data + NamesAndTypes schema; // TODO scchema info is from tipb + auto physical_exchange_receiver = std::make_shared( + context, + executor_id, + schema, + fine_grained_shuffle, + log->identifier(), + Block(schema)); + return physical_exchange_receiver; +} + void PhysicalCTESource::buildPipelineExecGroupImpl( PipelineExecutorContext & exec_context, PipelineExecGroupBuilder & group_builder, @@ -26,15 +52,13 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( if (fine_grained_shuffle.enabled()) concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); + std::shared_ptr cte; // TODO CTEManager: get it from CTEManager + for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) { - // group_builder.addConcurrency(std::make_unique( - // exec_context, - // log->identifier(), - // mpp_exchange_receiver, - // /*stream_id=*/fine_grained_shuffle.enabled() ? partition_id : 0)); + group_builder.addConcurrency(std::make_unique(exec_context, log->identifier(), cte)); } - // context.getDAGContext()->addInboundIOProfileInfos(executor_id, group_builder.getCurIOProfileInfos()); + context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } void PhysicalCTESource::finalizeImpl(const Names & parent_require) From 77d76522c1ddb88c6ad818a84380e62210d52e51 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 3 Apr 2025 19:03:05 +0800 Subject: [PATCH 010/118] save --- dbms/src/Common/TiFlashMetrics.h | 2 ++ dbms/src/Flash/Executor/PipelineExecutor.cpp | 32 +++++++++++-------- dbms/src/Flash/Executor/PipelineExecutor.h | 3 +- .../Flash/Executor/PipelineExecutorContext.h | 6 ++++ dbms/src/Flash/FlashService.cpp | 2 ++ dbms/src/Flash/Mpp/MPPTaskId.cpp | 5 +++ dbms/src/Flash/Mpp/MPPTaskId.h | 11 +++++-- dbms/src/Flash/Planner/PhysicalPlan.cpp | 10 ++++++ dbms/src/Flash/Planner/PhysicalPlan.h | 11 +++++-- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 15 +++++---- .../Flash/Planner/Plans/PhysicalCTESource.h | 16 +++++++--- dbms/src/Interpreters/Context.cpp | 4 +++ dbms/src/Operators/CTESink.Op.cpp | 8 +++-- dbms/src/Operators/CTESinkOp.h | 13 ++++++-- dbms/src/Operators/CTESource.cpp | 12 +++++-- dbms/src/Operators/CTESource.h | 16 +++++++--- 16 files changed, 123 insertions(+), 43 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 788d5a6f954..a0cf7fc76d5 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -95,6 +95,8 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_exchange_receiver, {"type", "exchange_receiver"}), \ F(type_projection, {"type", "projection"}), \ F(type_partition_ts, {"type", "partition_table_scan"}), \ + F(type_cte_sink, {"type", "cte_sink"}), \ + F(type_cte_source, {"type", "cte_source"}), \ F(type_window, {"type", "window"}), \ F(type_window_sort, {"type", "window_sort"}), \ F(type_expand, {"type", "expand"})) \ diff --git a/dbms/src/Flash/Executor/PipelineExecutor.cpp b/dbms/src/Flash/Executor/PipelineExecutor.cpp index f80cb1200e0..303db6273db 100644 --- a/dbms/src/Flash/Executor/PipelineExecutor.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutor.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -29,28 +30,31 @@ PipelineExecutor::PipelineExecutor( Context & context_, const String & req_id) : QueryExecutor(memory_tracker_, context_, req_id) - , exec_context( + , exec_context_ptr(std::make_shared( // For mpp task, there is a unique identifier MPPTaskId, so MPPTaskId is used here as the query id of PipelineExecutor. // But for cop/batchCop, there is no such unique identifier, so an empty value is given here, indicating that the query id of PipelineExecutor is invalid. /*query_id=*/context.getDAGContext()->isMPPTask() ? context.getDAGContext()->getMPPTaskId().toString() : "", + /*query_id_for_cte=*/context.getDAGContext()->isMPPTask() + ? context.getDAGContext()->getMPPTaskId().getQueryIDForCTE() + : "", req_id, memory_tracker_, context.getDAGContext(), auto_spill_trigger, register_operator_spill_context, - context.getDAGContext()->getResourceGroupName()) + context.getDAGContext()->getResourceGroupName())) { - PhysicalPlan physical_plan{context, log->identifier()}; + PhysicalPlan physical_plan{context, log->identifier(), exec_context_ptr}; physical_plan.build(context.getDAGContext()->dag_request()); physical_plan.outputAndOptimize(); - root_pipeline = physical_plan.toPipeline(exec_context, context); + root_pipeline = physical_plan.toPipeline(*exec_context_ptr, context); LocalAdmissionController::global_instance->warmupResourceGroupInfoCache(dagContext().getResourceGroupName()); } void PipelineExecutor::scheduleEvents() { assert(root_pipeline); - auto events = root_pipeline->toEvents(exec_context, context, context.getMaxStreams()); + auto events = root_pipeline->toEvents(*exec_context_ptr, context, context.getMaxStreams()); Events sources; for (const auto & event : events) { @@ -67,18 +71,18 @@ void PipelineExecutor::wait() { // In test mode, a single query should take no more than 5 minutes to execute. static std::chrono::minutes timeout(5); - exec_context.waitFor(timeout); + exec_context_ptr->waitFor(timeout); } else { - exec_context.wait(); + exec_context_ptr->wait(); } } void PipelineExecutor::consume(ResultHandler & result_handler) { assert(result_handler); - exec_context.consume(result_handler); + exec_context_ptr->consume(result_handler); } ExecutionResult PipelineExecutor::execute(ResultHandler && result_handler) @@ -91,7 +95,7 @@ ExecutionResult PipelineExecutor::execute(ResultHandler && result_handler) // The queue size is same as UnionBlockInputStream = concurrency * 5. assert(root_pipeline); - root_pipeline->addGetResultSink(exec_context.toConsumeMode(/*queue_size=*/context.getMaxStreams() * 5)); + root_pipeline->addGetResultSink(exec_context_ptr->toConsumeMode(/*queue_size=*/context.getMaxStreams() * 5)); scheduleEvents(); consume(result_handler); } @@ -100,13 +104,13 @@ ExecutionResult PipelineExecutor::execute(ResultHandler && result_handler) scheduleEvents(); wait(); } - LOG_DEBUG(log, "query finish with {}", exec_context.getQueryProfileInfo().toJson()); - return exec_context.toExecutionResult(); + LOG_DEBUG(log, "query finish with {}", exec_context_ptr->getQueryProfileInfo().toJson()); + return exec_context_ptr->toExecutionResult(); } void PipelineExecutor::cancel() { - exec_context.cancel(); + exec_context_ptr->cancel(); } String PipelineExecutor::toString() const @@ -129,7 +133,7 @@ UInt64 PipelineExecutor::collectCPUTimeNs() // Therefore, `query_profile_info.getCPUExecuteTimeNs()` is approximately equal to the actual CPU time of the query. // However, once these two assumptions are broken, it will lead to inaccurate acquisition of CPU time. // It may be necessary to obtain CPU time using a more accurate method, such as using system call `clock_gettime`. - const auto & query_profile_info = exec_context.getQueryProfileInfo(); + const auto & query_profile_info = exec_context_ptr->getQueryProfileInfo(); auto cpu_time_ns = query_profile_info.getCPUExecuteTimeNs(); return cpu_time_ns; } @@ -156,6 +160,6 @@ BaseRuntimeStatistics PipelineExecutor::getRuntimeStatistics() const String PipelineExecutor::getExtraJsonInfo() const { - return exec_context.getQueryProfileInfo().toJson(); + return exec_context_ptr->getQueryProfileInfo().toJson(); } } // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutor.h b/dbms/src/Flash/Executor/PipelineExecutor.h index ebac7fe65dc..08e0f6a7ade 100644 --- a/dbms/src/Flash/Executor/PipelineExecutor.h +++ b/dbms/src/Flash/Executor/PipelineExecutor.h @@ -24,6 +24,7 @@ class Context; class Pipeline; using PipelinePtr = std::shared_ptr; +using PipelineExecutorContextPtr = std::shared_ptr; using Pipelines = std::vector; class AutoSpillTrigger; @@ -88,6 +89,6 @@ class PipelineExecutor : public QueryExecutor private: PipelinePtr root_pipeline; - PipelineExecutorContext exec_context; + PipelineExecutorContextPtr exec_context_ptr; }; } // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.h b/dbms/src/Flash/Executor/PipelineExecutorContext.h index efc3e355cae..7b1c8f42c89 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.h +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.h @@ -53,6 +53,7 @@ class PipelineExecutorContext : private boost::noncopyable PipelineExecutorContext( const String & query_id_, + const String & query_id_for_cte_, const String & req_id, const MemoryTrackerPtr & mem_tracker_, DAGContext * dag_context_ = nullptr, @@ -60,6 +61,7 @@ class PipelineExecutorContext : private boost::noncopyable const RegisterOperatorSpillContext & register_operator_spill_context_ = nullptr, const String & resource_group_name_ = "") : query_id(query_id_) + , query_id_for_cte(query_id_for_cte_) , log(Logger::get(req_id)) , mem_tracker(mem_tracker_) , dag_context(dag_context_) @@ -114,6 +116,8 @@ class PipelineExecutorContext : private boost::noncopyable const String & getQueryId() const { return query_id; } + const String & getQueryIdForCTE() const { return this->query_id_for_cte; } + const MemoryTrackerPtr & getMemoryTracker() const { return mem_tracker; } void triggerAutoSpill() const @@ -158,6 +162,8 @@ class PipelineExecutorContext : private boost::noncopyable private: const String query_id; + String query_id_for_cte; + LoggerPtr log; MemoryTrackerPtr mem_tracker; diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 07af1390649..74305fd0a47 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -832,6 +832,8 @@ std::tuple FlashService::createDBContext(const grpc::S auto tmp_context = std::make_shared(*context); tmp_context->setGlobalContext(*context); + // TODO set cte manager here + /// Set a bunch of client information. std::string user = getClientMetaVarWithDefault(grpc_context, "user", "default"); std::string password = getClientMetaVarWithDefault(grpc_context, "password", ""); diff --git a/dbms/src/Flash/Mpp/MPPTaskId.cpp b/dbms/src/Flash/Mpp/MPPTaskId.cpp index 168ab7ac871..99ff83cd44d 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskId.cpp @@ -104,6 +104,11 @@ String MPPTaskId::toString() const : fmt::format("MPP<{},task_id:{}>", gather_id.toString(), task_id); } +String MPPTaskId::getQueryIDForCTE() const +{ + return isUnknown() ? "" : gather_id.toStringForQueryID(); +} + const MPPTaskId MPPTaskId::unknown_mpp_task_id = MPPTaskId{}; constexpr UInt64 MAX_UINT64 = std::numeric_limits::max(); diff --git a/dbms/src/Flash/Mpp/MPPTaskId.h b/dbms/src/Flash/Mpp/MPPTaskId.h index d9e0946424f..1b299378c1d 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.h +++ b/dbms/src/Flash/Mpp/MPPTaskId.h @@ -102,20 +102,24 @@ struct MPPGatherId : gather_id(gather_id_) , query_id(query_ts, local_query_id, server_id, start_ts, resource_group_name, connection_id, connection_alias) {} + explicit MPPGatherId(const mpp::TaskMeta & task_meta) : gather_id(task_meta.gather_id()) , query_id(task_meta) {} - String toString() const + + String toStringForQueryID() const { return fmt::format( - "gather_id:{}, query_ts:{}, local_query_id:{}, server_id:{}, start_ts:{}", - gather_id, + "query_ts:{}, local_query_id:{}, server_id:{}, start_ts:{}", query_id.query_ts, query_id.local_query_id, query_id.server_id, query_id.start_ts); } + + String toString() const { return fmt::format("gather_id:{}, {}", gather_id, this->toStringForQueryID()); } + bool hasMeaningfulGatherId() const { return gather_id > 0; } bool operator==(const MPPGatherId & rid) const; }; @@ -165,6 +169,7 @@ struct MPPTaskId bool isUnknown() const { return task_id == unknown_task_id; } String toString() const; + String getQueryIDForCTE() const; static const MPPTaskId unknown_mpp_task_id; static const MPPQueryId Max_Query_Id; diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index d841b5a1e8b..480c50e049c 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -41,6 +41,8 @@ #include #include +#include "Flash/Planner/Plans/PhysicalCTESource.h" + namespace DB { namespace @@ -238,7 +240,15 @@ void PhysicalPlan::build(const tipb::Executor * executor) GET_METRIC(tiflash_coprocessor_executor_count, type_expand).Increment(); pushBack(PhysicalExpand2::build(context, executor_id, log, executor->expand2(), popBack())); break; + + // TODO this is tmp code for tipb::ExecType::CTESource + GET_METRIC(tiflash_coprocessor_executor_count, type_cte_source).Increment(); + pushBack(PhysicalCTESource::build(context, executor_id, log, fine_grained_shuffle)); + // TODO this is tmp code for tipb::ExecType::CTESink + GET_METRIC(tiflash_coprocessor_executor_count, type_cte_sink).Increment(); } + // TODO add tipb::ExecType::CTESource + // TODO add tipb::ExecType::CTESink default: throw TiFlashException( fmt::format("{} executor is not supported", fmt::underlying(executor->tp())), diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h index 8c54295a750..7c661a1f730 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.h +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -17,23 +17,28 @@ #include #include #include +#include #include #include #include #include +#include + namespace DB { class Pipeline; using PipelinePtr = std::shared_ptr; +using PipelineExecutorContextPtr = std::shared_ptr; using Pipelines = std::vector; class PhysicalPlan { public: - PhysicalPlan(Context & context_, const String & req_id) + PhysicalPlan(Context & context_, const String & req_id, PipelineExecutorContextPtr pipeline_exec_ptr_) : context(context_) , log(Logger::get(req_id)) + , exec_context_ptr(pipeline_exec_ptr_) {} void build(const tipb::DAGRequest * dag_request); @@ -63,7 +68,7 @@ class PhysicalPlan void buildTableScan(const String & executor_id, const tipb::Executor * executor); private: - std::vector cur_plan_nodes{}; + std::vector cur_plan_nodes; // hold the root node of physical plan node tree after `outputAndOptimize`. PhysicalPlanNodePtr root_node; @@ -71,5 +76,7 @@ class PhysicalPlan Context & context; LoggerPtr log; + + PipelineExecutorContextPtr exec_context_ptr; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index ba4d1db3ec0..2c3401636c5 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -17,8 +17,7 @@ #include #include #include - -#include "Operators/CTESource.h" +#include namespace DB { @@ -26,11 +25,11 @@ PhysicalPlanNodePtr PhysicalCTESource::build( const Context & context, const String & executor_id, const LoggerPtr & log, - const FineGrainedShuffle & fine_grained_shuffle + const FineGrainedShuffle & fine_grained_shuffle, + PipelineExecutorContextPtr exec_context_ptr /* TODO tipb::ExchangeReceiver */) { // TODO tipb for cte: need output schema field in tipb for cte source - // TODO CTEManager: get cte from CTEManager // TODO we need to get meta data such as `partition_col_collators` for partitioning data NamesAndTypes schema; // TODO scchema info is from tipb auto physical_exchange_receiver = std::make_shared( @@ -39,7 +38,8 @@ PhysicalPlanNodePtr PhysicalCTESource::build( schema, fine_grained_shuffle, log->identifier(), - Block(schema)); + Block(schema), + exec_context_ptr); return physical_exchange_receiver; } @@ -52,11 +52,12 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( if (fine_grained_shuffle.enabled()) concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); - std::shared_ptr cte; // TODO CTEManager: get it from CTEManager + String query_id_and_cte_id_prefix = fmt::format("{}_{}", this->exec_context_ptr->getQueryIdForCTE(), this->cte_id); for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) { - group_builder.addConcurrency(std::make_unique(exec_context, log->identifier(), cte)); + group_builder.addConcurrency( + std::make_unique(exec_context, log->identifier(), this->exec_context_ptr->)); } context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h index 3cc83d04092..3b50299767c 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -14,11 +14,14 @@ #pragma once +#include +#include #include -#include namespace DB { +using PipelineExecutorContextPtr = std::shared_ptr; + class PhysicalCTESource : public PhysicalLeaf { public: @@ -26,7 +29,8 @@ class PhysicalCTESource : public PhysicalLeaf const Context & context, const String & executor_id, const LoggerPtr & log, - const FineGrainedShuffle & fine_grained_shuffle); + const FineGrainedShuffle & fine_grained_shuffle, + PipelineExecutorContextPtr exec_context_ptr); // TODO to partition data, we may need to call `ExchangeSenderInterpreterHelper::genPartitionColCollators` like PhysicalExchangeSender PhysicalCTESource( @@ -34,9 +38,11 @@ class PhysicalCTESource : public PhysicalLeaf const NamesAndTypes & schema_, const FineGrainedShuffle & fine_grained_shuffle, const String & req_id, - const Block & sample_block_) + const Block & sample_block_, + PipelineExecutorContextPtr exec_context_ptr_) : PhysicalLeaf(executor_id_, PlanType::CTESource, schema_, fine_grained_shuffle, req_id) , sample_block(sample_block_) + , exec_context_ptr(exec_context_ptr_) {} void finalizeImpl(const Names & parent_require) override; @@ -44,7 +50,6 @@ class PhysicalCTESource : public PhysicalLeaf const Block & getSampleBlock() const override; private: - // TODO void buildPipelineExecGroupImpl( PipelineExecutorContext & exec_context, PipelineExecGroupBuilder & group_builder, @@ -54,6 +59,7 @@ class PhysicalCTESource : public PhysicalLeaf private: Block sample_block; String query_id_and_cte_id; - std::shared_ptr cte; + UInt64 cte_id = 0; // TODO initialize it from tipb + PipelineExecutorContextPtr exec_context_ptr; }; } // namespace DB diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 1ebd4d5a19d..0324c135da6 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -76,6 +77,7 @@ #include #include +#include #include #include @@ -228,6 +230,8 @@ struct ContextShared std::shared_ptr shared_block_schemas; + std::unique_ptr cte_manager; + ContextShared( std::shared_ptr runtime_components_factory_, Context::ApplicationType app_type) diff --git a/dbms/src/Operators/CTESink.Op.cpp b/dbms/src/Operators/CTESink.Op.cpp index 044b215b341..a450062fdb4 100644 --- a/dbms/src/Operators/CTESink.Op.cpp +++ b/dbms/src/Operators/CTESink.Op.cpp @@ -13,13 +13,17 @@ // limitations under the License. #include - -#include "Operators/Operator.h" +#include namespace DB { void CTESinkOp::operateSuffixImpl() { + this->cte.reset(); + this->cte_manager->releaseCTE(this->query_id_and_cte_id); + + // In case some tasks are still in WAITING_FOR_NOTIFY status + this->cte->notifyEOF(); LOG_DEBUG(log, "finish write with {} rows", this->total_rows); } diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index a69f552a15f..eaae2624b38 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include @@ -22,9 +23,15 @@ namespace DB class CTESinkOp : public SinkOp { public: - CTESinkOp(PipelineExecutorContext & exec_context_, const String & req_id, std::shared_ptr cte_) + CTESinkOp( + PipelineExecutorContext & exec_context_, + const String & req_id, + const String & query_id_and_cte_id_, + CTEManager * cte_manager_) : SinkOp(exec_context_, req_id) - , cte(cte_) + , query_id_and_cte_id(query_id_and_cte_id_) + , cte_manager(cte_manager_) + , cte(cte_manager_->getCTE(query_id_and_cte_id_)) {} String getName() const override { return "CTESinkOp"; } @@ -35,6 +42,8 @@ class CTESinkOp : public SinkOp OperatorStatus writeImpl(Block && block) override; private: + String query_id_and_cte_id; + CTEManager * cte_manager; std::shared_ptr cte; size_t total_rows = 0; bool input_done = false; diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index 1e6e9df5ae5..a18fba69b3f 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -12,13 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include - -#include "Operators/CTE.h" -#include "Operators/Operator.h" +#include namespace DB { +void CTESourceOp::operateSuffixImpl() +{ + this->cte.reset(); + this->cte_manager->releaseCTE(this->query_id_and_cte_id); + LOG_DEBUG(log, "finish read {} rows from cte source", total_rows); +} + // TODO in some cases, source needs to manually filter some data when cte saves all data OperatorStatus CTESourceOp::readImpl(Block & block) { diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 2d7c212bc0c..6a24a950961 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -16,31 +16,39 @@ #include #include +#include #include #include #include namespace DB { -// TODO consider fine graine at compiling stage class CTESourceOp : public SourceOp { public: - CTESourceOp(PipelineExecutorContext & exec_context_, const String & req_id, std::shared_ptr cte_) + CTESourceOp( + PipelineExecutorContext & exec_context_, + const String & req_id, + const String & query_id_and_cte_id_, + CTEManager * cte_manager_) : SourceOp(exec_context_, req_id) - , cte(cte_) + , query_id_and_cte_id(query_id_and_cte_id_) + , cte_manager(cte_manager_) + , cte(cte_manager_->getCTE(query_id_and_cte_id_)) {} String getName() const override { return "CTESourceOp"; } protected: - void operateSuffixImpl() override { LOG_DEBUG(log, "finish read {} rows from cte source", total_rows); } + void operateSuffixImpl() override; OperatorStatus readImpl(Block & block) override; OperatorStatus awaitImpl() override; private: + String query_id_and_cte_id; + CTEManager * cte_manager; std::shared_ptr cte; size_t block_fetch_idx = 0; From 15af97fcf3eb28ddc998a38af48b4e06362799af Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 7 Apr 2025 13:24:26 +0800 Subject: [PATCH 011/118] save --- dbms/src/Flash/Planner/PhysicalPlan.cpp | 21 ++++++++++++------- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 12 ++++++----- .../src/Flash/Planner/Plans/PhysicalCTESink.h | 2 ++ .../Flash/Planner/Plans/PhysicalCTESource.cpp | 16 +++++++------- .../Flash/Planner/Plans/PhysicalCTESource.h | 8 ++----- dbms/src/Interpreters/Context.cpp | 8 ++++++- dbms/src/Interpreters/Context.h | 4 ++++ 7 files changed, 45 insertions(+), 26 deletions(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 480c50e049c..48dc61191e2 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -21,6 +21,8 @@ #include #include #include +#include +#include #include #include #include @@ -41,8 +43,6 @@ #include #include -#include "Flash/Planner/Plans/PhysicalCTESource.h" - namespace DB { namespace @@ -241,11 +241,18 @@ void PhysicalPlan::build(const tipb::Executor * executor) pushBack(PhysicalExpand2::build(context, executor_id, log, executor->expand2(), popBack())); break; - // TODO this is tmp code for tipb::ExecType::CTESource - GET_METRIC(tiflash_coprocessor_executor_count, type_cte_source).Increment(); - pushBack(PhysicalCTESource::build(context, executor_id, log, fine_grained_shuffle)); - // TODO this is tmp code for tipb::ExecType::CTESink - GET_METRIC(tiflash_coprocessor_executor_count, type_cte_sink).Increment(); + { + // This is tmp code for tipb::ExecType::CTESource + auto fine_grained_shuffle = FineGrainedShuffle(executor); + GET_METRIC(tiflash_coprocessor_executor_count, type_cte_source).Increment(); + pushBack(PhysicalCTESource::build(context, executor_id, log, fine_grained_shuffle)); + } + { + // This is tmp code for tipb::ExecType::CTESink + auto fine_grained_shuffle = FineGrainedShuffle(executor); + GET_METRIC(tiflash_coprocessor_executor_count, type_cte_sink).Increment(); + pushBack(PhysicalCTESink::build(executor_id, log, fine_grained_shuffle, popBack())); + } } // TODO add tipb::ExecType::CTESource // TODO add tipb::ExecType::CTESink diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index b62130932cd..6b7e5d4ebb4 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include @@ -27,7 +28,6 @@ PhysicalPlanNodePtr PhysicalCTESink::build( const FineGrainedShuffle & fine_grained_shuffle, const PhysicalPlanNodePtr & child) { - // TODO CTEManager: get cte from CTEManager RUNTIME_CHECK(child); auto physical_cte_sink = std::make_shared( @@ -43,14 +43,16 @@ PhysicalPlanNodePtr PhysicalCTESink::build( void PhysicalCTESink::buildPipelineExecGroupImpl( PipelineExecutorContext & exec_context, PipelineExecGroupBuilder & group_builder, - Context & /*context*/, + Context & context, size_t /*concurrency*/) { size_t partition_id = 0; + String query_id_and_cte_id_prefix = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); group_builder.transform([&](auto & builder) { - std::shared_ptr cte; // TODO CTEManager: get it from CTEManager - builder.setSinkOp(std::make_unique(exec_context, log->identifier(), cte)); - ++partition_id; + String query_id_and_cte_id = fmt::format("{}_{}", query_id_and_cte_id_prefix, partition_id); + builder.setSinkOp( + std::make_unique(exec_context, log->identifier(), query_id_and_cte_id, context.getCTEManager())); + partition_id++; }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h index 75376aafe95..049d0933fd6 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h @@ -49,5 +49,7 @@ class PhysicalCTESink : public PhysicalUnary PipelineExecGroupBuilder & group_builder, Context & context, size_t /*concurrency*/) override; + + UInt64 cte_id = 0; // TODO initialize it from tipb }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 2c3401636c5..9a34e5dee0f 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -25,8 +25,7 @@ PhysicalPlanNodePtr PhysicalCTESource::build( const Context & context, const String & executor_id, const LoggerPtr & log, - const FineGrainedShuffle & fine_grained_shuffle, - PipelineExecutorContextPtr exec_context_ptr + const FineGrainedShuffle & fine_grained_shuffle /* TODO tipb::ExchangeReceiver */) { // TODO tipb for cte: need output schema field in tipb for cte source @@ -38,8 +37,7 @@ PhysicalPlanNodePtr PhysicalCTESource::build( schema, fine_grained_shuffle, log->identifier(), - Block(schema), - exec_context_ptr); + Block(schema)); return physical_exchange_receiver; } @@ -52,12 +50,16 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( if (fine_grained_shuffle.enabled()) concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); - String query_id_and_cte_id_prefix = fmt::format("{}_{}", this->exec_context_ptr->getQueryIdForCTE(), this->cte_id); + String query_id_and_cte_id_prefix = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) { - group_builder.addConcurrency( - std::make_unique(exec_context, log->identifier(), this->exec_context_ptr->)); + String query_id_and_cte_id = fmt::format("{}_{}", query_id_and_cte_id_prefix, partition_id); + group_builder.addConcurrency(std::make_unique( + exec_context, + log->identifier(), + query_id_and_cte_id, + context.getCTEManager())); } context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h index 3b50299767c..e9daa6959a3 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -29,8 +29,7 @@ class PhysicalCTESource : public PhysicalLeaf const Context & context, const String & executor_id, const LoggerPtr & log, - const FineGrainedShuffle & fine_grained_shuffle, - PipelineExecutorContextPtr exec_context_ptr); + const FineGrainedShuffle & fine_grained_shuffle); // TODO to partition data, we may need to call `ExchangeSenderInterpreterHelper::genPartitionColCollators` like PhysicalExchangeSender PhysicalCTESource( @@ -38,11 +37,9 @@ class PhysicalCTESource : public PhysicalLeaf const NamesAndTypes & schema_, const FineGrainedShuffle & fine_grained_shuffle, const String & req_id, - const Block & sample_block_, - PipelineExecutorContextPtr exec_context_ptr_) + const Block & sample_block_) : PhysicalLeaf(executor_id_, PlanType::CTESource, schema_, fine_grained_shuffle, req_id) , sample_block(sample_block_) - , exec_context_ptr(exec_context_ptr_) {} void finalizeImpl(const Names & parent_require) override; @@ -60,6 +57,5 @@ class PhysicalCTESource : public PhysicalLeaf Block sample_block; String query_id_and_cte_id; UInt64 cte_id = 0; // TODO initialize it from tipb - PipelineExecutorContextPtr exec_context_ptr; }; } // namespace DB diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 0324c135da6..fcd00f89925 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -230,7 +230,7 @@ struct ContextShared std::shared_ptr shared_block_schemas; - std::unique_ptr cte_manager; + std::unique_ptr cte_manager; // TODO initialize it ContextShared( std::shared_ptr runtime_components_factory_, @@ -574,6 +574,12 @@ PathPool & Context::getPathPool() const return shared->path_pool; } +CTEManager * Context::getCTEManager() const +{ + auto lock = getLock(); + return this->shared->cte_manager.get(); +} + void Context::setPath(const String & path) { auto lock = getLock(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 4f346ee2de8..1cbb9b6c876 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -36,6 +36,8 @@ #include #include +#include "Flash/Mpp/CTEManager.h" + namespace pingcap { @@ -214,6 +216,8 @@ class Context String getUserFilesPath() const; PathPool & getPathPool() const; + CTEManager * getCTEManager() const; + void setPath(const String & path); void setTemporaryPath(const String & path); void setFlagsPath(const String & path); From aa01ff60e987023e9fa9db5e5f47588824072bcf Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 7 Apr 2025 16:41:58 +0800 Subject: [PATCH 012/118] tweaking --- dbms/src/Flash/Executor/PipelineExecutor.cpp | 28 +++++++++---------- dbms/src/Flash/Executor/PipelineExecutor.h | 3 +- dbms/src/Flash/Planner/PhysicalPlan.h | 6 +--- .../Flash/Planner/Plans/PhysicalCTESource.h | 2 -- dbms/src/Interpreters/Context.cpp | 3 +- dbms/src/Interpreters/Context.h | 4 +-- dbms/src/Operators/CTE.cpp | 10 ++----- dbms/src/Operators/CTE.h | 4 +-- 8 files changed, 22 insertions(+), 38 deletions(-) diff --git a/dbms/src/Flash/Executor/PipelineExecutor.cpp b/dbms/src/Flash/Executor/PipelineExecutor.cpp index 303db6273db..1f8104234ac 100644 --- a/dbms/src/Flash/Executor/PipelineExecutor.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutor.cpp @@ -30,7 +30,7 @@ PipelineExecutor::PipelineExecutor( Context & context_, const String & req_id) : QueryExecutor(memory_tracker_, context_, req_id) - , exec_context_ptr(std::make_shared( + , exec_context( // For mpp task, there is a unique identifier MPPTaskId, so MPPTaskId is used here as the query id of PipelineExecutor. // But for cop/batchCop, there is no such unique identifier, so an empty value is given here, indicating that the query id of PipelineExecutor is invalid. /*query_id=*/context.getDAGContext()->isMPPTask() ? context.getDAGContext()->getMPPTaskId().toString() : "", @@ -42,19 +42,19 @@ PipelineExecutor::PipelineExecutor( context.getDAGContext(), auto_spill_trigger, register_operator_spill_context, - context.getDAGContext()->getResourceGroupName())) + context.getDAGContext()->getResourceGroupName()) { - PhysicalPlan physical_plan{context, log->identifier(), exec_context_ptr}; + PhysicalPlan physical_plan{context, log->identifier()}; physical_plan.build(context.getDAGContext()->dag_request()); physical_plan.outputAndOptimize(); - root_pipeline = physical_plan.toPipeline(*exec_context_ptr, context); + root_pipeline = physical_plan.toPipeline(exec_context, context); LocalAdmissionController::global_instance->warmupResourceGroupInfoCache(dagContext().getResourceGroupName()); } void PipelineExecutor::scheduleEvents() { assert(root_pipeline); - auto events = root_pipeline->toEvents(*exec_context_ptr, context, context.getMaxStreams()); + auto events = root_pipeline->toEvents(exec_context, context, context.getMaxStreams()); Events sources; for (const auto & event : events) { @@ -71,18 +71,18 @@ void PipelineExecutor::wait() { // In test mode, a single query should take no more than 5 minutes to execute. static std::chrono::minutes timeout(5); - exec_context_ptr->waitFor(timeout); + exec_context.waitFor(timeout); } else { - exec_context_ptr->wait(); + exec_context.wait(); } } void PipelineExecutor::consume(ResultHandler & result_handler) { assert(result_handler); - exec_context_ptr->consume(result_handler); + exec_context.consume(result_handler); } ExecutionResult PipelineExecutor::execute(ResultHandler && result_handler) @@ -95,7 +95,7 @@ ExecutionResult PipelineExecutor::execute(ResultHandler && result_handler) // The queue size is same as UnionBlockInputStream = concurrency * 5. assert(root_pipeline); - root_pipeline->addGetResultSink(exec_context_ptr->toConsumeMode(/*queue_size=*/context.getMaxStreams() * 5)); + root_pipeline->addGetResultSink(exec_context.toConsumeMode(/*queue_size=*/context.getMaxStreams() * 5)); scheduleEvents(); consume(result_handler); } @@ -104,13 +104,13 @@ ExecutionResult PipelineExecutor::execute(ResultHandler && result_handler) scheduleEvents(); wait(); } - LOG_DEBUG(log, "query finish with {}", exec_context_ptr->getQueryProfileInfo().toJson()); - return exec_context_ptr->toExecutionResult(); + LOG_DEBUG(log, "query finish with {}", exec_context.getQueryProfileInfo().toJson()); + return exec_context.toExecutionResult(); } void PipelineExecutor::cancel() { - exec_context_ptr->cancel(); + exec_context.cancel(); } String PipelineExecutor::toString() const @@ -133,7 +133,7 @@ UInt64 PipelineExecutor::collectCPUTimeNs() // Therefore, `query_profile_info.getCPUExecuteTimeNs()` is approximately equal to the actual CPU time of the query. // However, once these two assumptions are broken, it will lead to inaccurate acquisition of CPU time. // It may be necessary to obtain CPU time using a more accurate method, such as using system call `clock_gettime`. - const auto & query_profile_info = exec_context_ptr->getQueryProfileInfo(); + const auto & query_profile_info = exec_context.getQueryProfileInfo(); auto cpu_time_ns = query_profile_info.getCPUExecuteTimeNs(); return cpu_time_ns; } @@ -160,6 +160,6 @@ BaseRuntimeStatistics PipelineExecutor::getRuntimeStatistics() const String PipelineExecutor::getExtraJsonInfo() const { - return exec_context_ptr->getQueryProfileInfo().toJson(); + return exec_context.getQueryProfileInfo().toJson(); } } // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutor.h b/dbms/src/Flash/Executor/PipelineExecutor.h index 08e0f6a7ade..ebac7fe65dc 100644 --- a/dbms/src/Flash/Executor/PipelineExecutor.h +++ b/dbms/src/Flash/Executor/PipelineExecutor.h @@ -24,7 +24,6 @@ class Context; class Pipeline; using PipelinePtr = std::shared_ptr; -using PipelineExecutorContextPtr = std::shared_ptr; using Pipelines = std::vector; class AutoSpillTrigger; @@ -89,6 +88,6 @@ class PipelineExecutor : public QueryExecutor private: PipelinePtr root_pipeline; - PipelineExecutorContextPtr exec_context_ptr; + PipelineExecutorContext exec_context; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h index 7c661a1f730..02a12853d1d 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.h +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -29,16 +29,14 @@ namespace DB { class Pipeline; using PipelinePtr = std::shared_ptr; -using PipelineExecutorContextPtr = std::shared_ptr; using Pipelines = std::vector; class PhysicalPlan { public: - PhysicalPlan(Context & context_, const String & req_id, PipelineExecutorContextPtr pipeline_exec_ptr_) + PhysicalPlan(Context & context_, const String & req_id) : context(context_) , log(Logger::get(req_id)) - , exec_context_ptr(pipeline_exec_ptr_) {} void build(const tipb::DAGRequest * dag_request); @@ -76,7 +74,5 @@ class PhysicalPlan Context & context; LoggerPtr log; - - PipelineExecutorContextPtr exec_context_ptr; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h index e9daa6959a3..eb22fbe64e6 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -20,8 +20,6 @@ namespace DB { -using PipelineExecutorContextPtr = std::shared_ptr; - class PhysicalCTESource : public PhysicalLeaf { public: diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index fcd00f89925..6a967c981b0 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -230,7 +230,7 @@ struct ContextShared std::shared_ptr shared_block_schemas; - std::unique_ptr cte_manager; // TODO initialize it + std::unique_ptr cte_manager; ContextShared( std::shared_ptr runtime_components_factory_, @@ -238,6 +238,7 @@ struct ContextShared : runtime_components_factory(std::move(runtime_components_factory_)) , storage_run_mode(PageStorageRunMode::ONLY_V3) , application_type(app_type) + , cte_manager(std::make_unique()) { /// TODO: make it singleton (?) #ifndef MULTIPLE_CONTEXT_GTEST diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 1cbb9b6c876..b35aa422058 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -36,9 +37,6 @@ #include #include -#include "Flash/Mpp/CTEManager.h" - - namespace pingcap { struct ClusterConfig; diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 8e0584ee32f..3569013e8a7 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - #include #include @@ -24,7 +23,7 @@ namespace DB FetchStatus CTE::checkAvailableBlockAt(size_t idx) { std::shared_lock lock(this->rw_lock); - auto block_num = this->blocks.size(); // TODO consider spill + auto block_num = this->blocks.size(); if (block_num <= idx) { if (this->is_eof) @@ -32,14 +31,13 @@ FetchStatus CTE::checkAvailableBlockAt(size_t idx) else return FetchStatus::Waiting; } - // TODO handle FetchStatus::Cancelled return FetchStatus::Ok; } std::pair CTE::tryGetBlockAt(size_t idx) { std::shared_lock lock(this->rw_lock); - auto block_num = this->blocks.size(); // TODO maybe blocks are in disk + auto block_num = this->blocks.size(); if (block_num <= idx) { if (this->is_eof) @@ -47,8 +45,6 @@ std::pair CTE::tryGetBlockAt(size_t idx) else return {FetchStatus::Waiting, Block()}; } - // TODO handle error and cancel - // TODO maybe fetch block from disk return {FetchStatus::Ok, this->blocks[idx]}; } @@ -56,9 +52,7 @@ void CTE::pushBlock(const Block & block) { std::unique_lock lock(this->rw_lock); - // TODO consider spill this->memory_usage += block.bytes(); - // TODO check spill if unlikely (this->blocks.empty()) this->pipe_cv.notifyAll(); this->blocks.push_back(block); diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 13adcc998f1..a32a3bef240 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -47,7 +47,7 @@ class CTE : public NotifyFuture private: // Return true if CTE has data - inline bool hasDataNoLock() const { return !this->blocks.empty() || this->spill_triggered; } + inline bool hasDataNoLock() const { return !this->blocks.empty(); } std::shared_mutex rw_lock; Blocks blocks; @@ -59,7 +59,5 @@ class CTE : public NotifyFuture PipeConditionVariable pipe_cv; bool is_eof = false; - bool spill_triggered = false; // TODO this var may be useless, just a placement so far - // TODO spill }; } // namespace DB From 218f0ed92430221898b42da662be6e8ff39412d0 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 7 Apr 2025 17:32:41 +0800 Subject: [PATCH 013/118] tweaking --- dbms/src/Flash/FlashService.cpp | 2 -- dbms/src/Flash/Mpp/MPPTaskId.cpp | 5 ----- dbms/src/Flash/Mpp/MPPTaskId.h | 6 +++--- dbms/src/Flash/Planner/PhysicalPlan.h | 3 --- dbms/src/Operators/CTE.h | 1 - dbms/src/Operators/{CTESink.Op.cpp => CTESinkOp.cpp} | 2 +- 6 files changed, 4 insertions(+), 15 deletions(-) rename dbms/src/Operators/{CTESink.Op.cpp => CTESinkOp.cpp} (95%) diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 74305fd0a47..07af1390649 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -832,8 +832,6 @@ std::tuple FlashService::createDBContext(const grpc::S auto tmp_context = std::make_shared(*context); tmp_context->setGlobalContext(*context); - // TODO set cte manager here - /// Set a bunch of client information. std::string user = getClientMetaVarWithDefault(grpc_context, "user", "default"); std::string password = getClientMetaVarWithDefault(grpc_context, "password", ""); diff --git a/dbms/src/Flash/Mpp/MPPTaskId.cpp b/dbms/src/Flash/Mpp/MPPTaskId.cpp index 99ff83cd44d..168ab7ac871 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskId.cpp @@ -104,11 +104,6 @@ String MPPTaskId::toString() const : fmt::format("MPP<{},task_id:{}>", gather_id.toString(), task_id); } -String MPPTaskId::getQueryIDForCTE() const -{ - return isUnknown() ? "" : gather_id.toStringForQueryID(); -} - const MPPTaskId MPPTaskId::unknown_mpp_task_id = MPPTaskId{}; constexpr UInt64 MAX_UINT64 = std::numeric_limits::max(); diff --git a/dbms/src/Flash/Mpp/MPPTaskId.h b/dbms/src/Flash/Mpp/MPPTaskId.h index 1b299378c1d..6005d5b872f 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.h +++ b/dbms/src/Flash/Mpp/MPPTaskId.h @@ -108,7 +108,7 @@ struct MPPGatherId , query_id(task_meta) {} - String toStringForQueryID() const + inline String toStringForQueryID() const { return fmt::format( "query_ts:{}, local_query_id:{}, server_id:{}, start_ts:{}", @@ -118,7 +118,7 @@ struct MPPGatherId query_id.start_ts); } - String toString() const { return fmt::format("gather_id:{}, {}", gather_id, this->toStringForQueryID()); } + inline String toString() const { return fmt::format("gather_id:{}, {}", gather_id, this->toStringForQueryID()); } bool hasMeaningfulGatherId() const { return gather_id > 0; } bool operator==(const MPPGatherId & rid) const; @@ -168,8 +168,8 @@ struct MPPTaskId bool isUnknown() const { return task_id == unknown_task_id; } + inline String getQueryIDForCTE() const { return isUnknown() ? "" : gather_id.toStringForQueryID(); } String toString() const; - String getQueryIDForCTE() const; static const MPPTaskId unknown_mpp_task_id; static const MPPQueryId Max_Query_Id; diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h index 02a12853d1d..e94ca85631d 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.h +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -17,14 +17,11 @@ #include #include #include -#include #include #include #include #include -#include - namespace DB { class Pipeline; diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index a32a3bef240..18d05626958 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -32,7 +32,6 @@ enum class FetchStatus Cancelled }; -// TODO track memory with allocator class CTE : public NotifyFuture { public: diff --git a/dbms/src/Operators/CTESink.Op.cpp b/dbms/src/Operators/CTESinkOp.cpp similarity index 95% rename from dbms/src/Operators/CTESink.Op.cpp rename to dbms/src/Operators/CTESinkOp.cpp index a450062fdb4..39eebf16917 100644 --- a/dbms/src/Operators/CTESink.Op.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -36,7 +36,7 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) return OperatorStatus::FINISHED; } this->total_rows += block.rows(); - this->cte->pushBlock(block); // TODO handle spill + this->cte->pushBlock(block); return OperatorStatus::NEED_INPUT; } } // namespace DB From eb04c29fa62fac4bf0542f6720e12141d4968f17 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 8 Apr 2025 14:53:18 +0800 Subject: [PATCH 014/118] tweaking --- dbms/src/Common/WeakHash.h | 2 +- dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp | 3 +-- dbms/src/Operators/CTESinkOp.h | 2 ++ dbms/src/Operators/CTESource.cpp | 4 +--- dbms/src/Operators/CTESource.h | 2 ++ 5 files changed, 7 insertions(+), 6 deletions(-) diff --git a/dbms/src/Common/WeakHash.h b/dbms/src/Common/WeakHash.h index 7f9214c4ed8..f7bf97f4e69 100644 --- a/dbms/src/Common/WeakHash.h +++ b/dbms/src/Common/WeakHash.h @@ -28,7 +28,7 @@ class WeakHash32 public: using Container = PaddedPODArray; - static constexpr UInt32 initial_hash = ~UInt32(0); + static constexpr UInt32 initial_hash = ~static_cast(0); explicit WeakHash32(size_t size) : data(size, initial_hash) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 9a34e5dee0f..0f78e1ada5a 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -22,7 +22,7 @@ namespace DB { PhysicalPlanNodePtr PhysicalCTESource::build( - const Context & context, + const Context & /*context*/, const String & executor_id, const LoggerPtr & log, const FineGrainedShuffle & fine_grained_shuffle @@ -32,7 +32,6 @@ PhysicalPlanNodePtr PhysicalCTESource::build( // TODO we need to get meta data such as `partition_col_collators` for partitioning data NamesAndTypes schema; // TODO scchema info is from tipb auto physical_exchange_receiver = std::make_shared( - context, executor_id, schema, fine_grained_shuffle, diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index eaae2624b38..632815d6b7e 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -34,6 +34,8 @@ class CTESinkOp : public SinkOp , cte(cte_manager_->getCTE(query_id_and_cte_id_)) {} + ~CTESinkOp() override { assert(!this->cte); } + String getName() const override { return "CTESinkOp"; } bool canHandleSelectiveBlock() const override { return true; } diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index a18fba69b3f..6f20a238e08 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -25,10 +25,9 @@ void CTESourceOp::operateSuffixImpl() LOG_DEBUG(log, "finish read {} rows from cte source", total_rows); } -// TODO in some cases, source needs to manually filter some data when cte saves all data +// TODO in some cases, source needs to manually filter some data when cte saves all data(start here) OperatorStatus CTESourceOp::readImpl(Block & block) { - // TODO when spill is triggered, this function may be blocked by spill. fix it auto res = this->cte->tryGetBlockAt(this->block_fetch_idx); switch (res.first) { @@ -51,7 +50,6 @@ OperatorStatus CTESourceOp::readImpl(Block & block) OperatorStatus CTESourceOp::awaitImpl() { - // TODO when spill is triggered, this function may be blocked by spill. fix it auto res = this->cte->checkAvailableBlockAt(this->block_fetch_idx); switch (res) { diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 6a24a950961..e089ceb4312 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -37,6 +37,8 @@ class CTESourceOp : public SourceOp , cte(cte_manager_->getCTE(query_id_and_cte_id_)) {} + ~CTESourceOp() override { assert(!this->cte); } + String getName() const override { return "CTESourceOp"; } protected: From c0d617a8ab71cd144b2e6fa77d633c1e826df47c Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 8 Apr 2025 15:26:28 +0800 Subject: [PATCH 015/118] fix --- .../src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp | 2 +- .../Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp | 4 ++-- .../Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp | 4 ++-- .../TaskQueues/tests/gtest_resource_control_queue.cpp | 5 ++++- .../Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp | 2 +- 5 files changed, 10 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp index 643153e2d30..6f30bbf52d5 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp @@ -519,7 +519,7 @@ CATCH TEST_F(EventTestRunner, memoryTrace) try { - PipelineExecutorContext exec_context{"", "", MemoryTracker::create()}; + PipelineExecutorContext exec_context{"", "", "", MemoryTracker::create()}; auto event = std::make_shared(exec_context); if (event->prepare()) event->schedule(); diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp index 398f5a4dff3..d71bc1187a4 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp @@ -180,12 +180,12 @@ CATCH TEST_F(TestIOPriorityTaskQueue, cancel) try { - PipelineExecutorContext context1("id1", "", nullptr); + PipelineExecutorContext context1("id1", "id1", "", nullptr); // To avoid the active ref count being returned to 0 in advance. context1.incActiveRefCount(); SCOPE_EXIT({ context1.decActiveRefCount(); }); - PipelineExecutorContext context2("id2", "", nullptr); + PipelineExecutorContext context2("id2", "id2", "", nullptr); // To avoid the active ref count being returned to 0 in advance. context2.incActiveRefCount(); SCOPE_EXIT({ context2.decActiveRefCount(); }); diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp index 6d004f956a4..3ca66205948 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp @@ -236,12 +236,12 @@ CATCH TEST_F(TestMLFQTaskQueue, cancel) try { - PipelineExecutorContext context1("id1", "", nullptr); + PipelineExecutorContext context1("id1", "id1", "", nullptr); // To avoid the active ref count being returned to 0 in advance. context1.incActiveRefCount(); SCOPE_EXIT({ context1.decActiveRefCount(); }); - PipelineExecutorContext context2("id2", "", nullptr); + PipelineExecutorContext context2("id2", "id2", "", nullptr); // To avoid the active ref count being returned to 0 in advance. context2.incActiveRefCount(); SCOPE_EXIT({ context2.decActiveRefCount(); }); diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp index 76565dac6ea..4b1bb87af32 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp @@ -171,6 +171,7 @@ class TestResourceControlQueue : public ::testing::Test { auto resource_group_name = resource_groups[i]->name; all_contexts[i] = std::make_shared( + query_id_prefix + resource_group_name, query_id_prefix + resource_group_name, req_id_prefix + resource_group_name, mem_tracker, @@ -374,6 +375,7 @@ TEST_F(TestResourceControlQueue, BasicTest) { String group_name = "rg" + std::to_string(i); all_contexts[i] = std::make_shared( + "mock-query-id", "mock-query-id", "mock-req-id", mem_tracker, @@ -410,6 +412,7 @@ TEST_F(TestResourceControlQueue, BasicTimeoutTest) // In the destructor of Task, will use PipelineExecutorContext to log. String group_name = "rg1"; auto exec_context = std::make_shared( + "mock-query-id", "mock-query-id", "mock-req-id", mem_tracker, @@ -445,7 +448,7 @@ TEST_F(TestResourceControlQueue, RunOutOfRU) TaskScheduler task_scheduler(config); PipelineExecutorContext - exec_context("mock-query-id", "mock-req-id", mem_tracker, nullptr, nullptr, nullptr, rg_name); + exec_context("mock-query-id", "mock-query-id", "mock-req-id", mem_tracker, nullptr, nullptr, nullptr, rg_name); auto task = std::make_unique(exec_context); // This task should use 5*100ms cpu_time. diff --git a/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp b/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp index c69a84b3a30..f613f2af88b 100644 --- a/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp @@ -224,7 +224,7 @@ try { for (size_t task_num = 1; task_num < 100; ++task_num) { - PipelineExecutorContext exec_context{"", "", MemoryTracker::create()}; + PipelineExecutorContext exec_context{"", "", "", MemoryTracker::create()}; std::vector tasks; for (size_t i = 0; i < task_num; ++i) tasks.push_back(std::make_unique(exec_context)); From c1512c479c1c2250fca74e637d47cccb8c6e698e Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 8 Apr 2025 17:01:22 +0800 Subject: [PATCH 016/118] tweaking --- .../ExchangeSenderInterpreterHelper.cpp | 22 ++++++++------- .../ExchangeSenderInterpreterHelper.h | 8 +++--- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 6 +++-- .../src/Flash/Planner/Plans/PhysicalCTESink.h | 9 ++++--- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 27 ++++++++++++++----- .../Flash/Planner/Plans/PhysicalCTESource.h | 15 ++++++++--- .../Planner/Plans/PhysicalExchangeSender.cpp | 8 +++--- dbms/src/Operators/CTESource.h | 9 ++++++- 8 files changed, 72 insertions(+), 32 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.cpp b/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.cpp index 0a858a785dd..e5d51963e6a 100644 --- a/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.cpp +++ b/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.cpp @@ -23,10 +23,10 @@ namespace DB::ExchangeSenderInterpreterHelper { -std::vector genPartitionColIds(const tipb::ExchangeSender & exchange_sender) +std::vector genPartitionColIds(const ::google::protobuf::RepeatedPtrField<::tipb::Expr> & partition_keys) { std::vector partition_col_ids; - for (const auto & part_key : exchange_sender.partition_keys()) + for (const auto & part_key : partition_keys) { if (unlikely(!isColumnExpr(part_key))) { @@ -41,13 +41,15 @@ std::vector genPartitionColIds(const tipb::ExchangeSender & exchange_send return partition_col_ids; } -TiDB::TiDBCollators genPartitionColCollators(const tipb::ExchangeSender & exchange_sender) +TiDB::TiDBCollators genPartitionColCollators( + const ::google::protobuf::RepeatedPtrField<::tipb::Expr> & partition_keys, + const ::google::protobuf::RepeatedPtrField<::tipb::FieldType> & types) { TiDB::TiDBCollators partition_col_collators; - const auto & part_keys = exchange_sender.partition_keys(); + auto type_num = types.size(); /// in case TiDB is an old version, it has no collation info - bool has_collator_info = exchange_sender.types_size() != 0; - if (unlikely(has_collator_info && part_keys.size() != exchange_sender.types_size())) + bool has_collator_info = type_num != 0; + if (unlikely(has_collator_info && partition_keys.size() != type_num)) { throw TiFlashException( fmt::format( @@ -56,12 +58,12 @@ TiDB::TiDBCollators genPartitionColCollators(const tipb::ExchangeSender & exchan __PRETTY_FUNCTION__), Errors::Coprocessor::BadRequest); } - for (int i = 0; i < part_keys.size(); ++i) + for (int i = 0; i < partition_keys.size(); ++i) { - const auto & expr = part_keys[i]; + const auto & expr = partition_keys[i]; if (has_collator_info && removeNullable(getDataTypeByFieldTypeForComputingLayer(expr.field_type()))->isString()) { - partition_col_collators.emplace_back(getCollatorFromFieldType(exchange_sender.types(i))); + partition_col_collators.emplace_back(getCollatorFromFieldType(types.at(i))); } else { @@ -70,4 +72,4 @@ TiDB::TiDBCollators genPartitionColCollators(const tipb::ExchangeSender & exchan } return partition_col_collators; } -} // namespace DB::ExchangeSenderInterpreterHelper \ No newline at end of file +} // namespace DB::ExchangeSenderInterpreterHelper diff --git a/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.h b/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.h index a7e980aac67..c1b88cf2d04 100644 --- a/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.h +++ b/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.h @@ -20,7 +20,9 @@ namespace DB::ExchangeSenderInterpreterHelper { -std::vector genPartitionColIds(const tipb::ExchangeSender & exchange_sender); +std::vector genPartitionColIds(const ::google::protobuf::RepeatedPtrField<::tipb::Expr> & partition_keys); -TiDB::TiDBCollators genPartitionColCollators(const tipb::ExchangeSender & exchange_sender); -} // namespace DB::ExchangeSenderInterpreterHelper \ No newline at end of file +TiDB::TiDBCollators genPartitionColCollators( + const ::google::protobuf::RepeatedPtrField<::tipb::Expr> & partition_keys, + const ::google::protobuf::RepeatedPtrField<::tipb::FieldType> & types); +} // namespace DB::ExchangeSenderInterpreterHelper diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 6b7e5d4ebb4..15e8bf0265d 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -26,7 +26,8 @@ PhysicalPlanNodePtr PhysicalCTESink::build( const String & executor_id, const LoggerPtr & log, const FineGrainedShuffle & fine_grained_shuffle, - const PhysicalPlanNodePtr & child) + const PhysicalPlanNodePtr & child, + UInt32 cte_id) { RUNTIME_CHECK(child); @@ -35,7 +36,8 @@ PhysicalPlanNodePtr PhysicalCTESink::build( child->getSchema(), fine_grained_shuffle, log->identifier(), - child); + child, + cte_id); physical_cte_sink->disableRestoreConcurrency(); return physical_cte_sink; } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h index 049d0933fd6..7f22406f97b 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h @@ -28,15 +28,18 @@ class PhysicalCTESink : public PhysicalUnary const String & executor_id, const LoggerPtr & log, const FineGrainedShuffle & fine_grained_shuffle, - const PhysicalPlanNodePtr & child); + const PhysicalPlanNodePtr & child, + UInt32 cte_id); PhysicalCTESink( const String & executor_id_, const NamesAndTypes & schema_, const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, - const PhysicalPlanNodePtr & child_) + const PhysicalPlanNodePtr & child_, + UInt32 cte_id_) : PhysicalUnary(executor_id_, PlanType::CTESink, schema_, fine_grained_shuffle_, req_id, child_) + , cte_id(cte_id_) {} void finalizeImpl(const Names & parent_require) override; @@ -50,6 +53,6 @@ class PhysicalCTESink : public PhysicalUnary Context & context, size_t /*concurrency*/) override; - UInt64 cte_id = 0; // TODO initialize it from tipb + UInt32 cte_id; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 0f78e1ada5a..7d716f2f76e 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -19,18 +20,30 @@ #include #include + namespace DB { PhysicalPlanNodePtr PhysicalCTESource::build( const Context & /*context*/, const String & executor_id, const LoggerPtr & log, - const FineGrainedShuffle & fine_grained_shuffle - /* TODO tipb::ExchangeReceiver */) + const FineGrainedShuffle & fine_grained_shuffle, + const tipb::CTESource & cte_source) { - // TODO tipb for cte: need output schema field in tipb for cte source - // TODO we need to get meta data such as `partition_col_collators` for partitioning data - NamesAndTypes schema; // TODO scchema info is from tipb + std::vector partition_col_ids + = ExchangeSenderInterpreterHelper::genPartitionColIds(cte_source.partition_keys()); + TiDB::TiDBCollators partition_col_collators + = ExchangeSenderInterpreterHelper::genPartitionColCollators(cte_source.partition_keys(), cte_source.types()); + + DAGSchema dag_schema; + for (int i = 0; i < cte_source.field_types_size(); ++i) + { + String name = genNameForExchangeReceiver(i); + TiDB::ColumnInfo info = TiDB::fieldTypeToColumnInfo(cte_source.field_types(i)); + dag_schema.emplace_back(std::move(name), std::move(info)); + } + + NamesAndTypes schema = toNamesAndTypes(dag_schema); auto physical_exchange_receiver = std::make_shared( executor_id, schema, @@ -58,7 +71,9 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( exec_context, log->identifier(), query_id_and_cte_id, - context.getCTEManager())); + context.getCTEManager(), + this->partition_col_ids, + this->partition_col_collators)); } context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h index eb22fbe64e6..aa9f90d9728 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -27,17 +28,21 @@ class PhysicalCTESource : public PhysicalLeaf const Context & context, const String & executor_id, const LoggerPtr & log, - const FineGrainedShuffle & fine_grained_shuffle); + const FineGrainedShuffle & fine_grained_shuffle, + const tipb::CTESource & cte_source); - // TODO to partition data, we may need to call `ExchangeSenderInterpreterHelper::genPartitionColCollators` like PhysicalExchangeSender PhysicalCTESource( const String & executor_id_, const NamesAndTypes & schema_, const FineGrainedShuffle & fine_grained_shuffle, const String & req_id, - const Block & sample_block_) + const Block & sample_block_, + const std::vector partition_col_ids_, + const TiDB::TiDBCollators partition_col_collators_) : PhysicalLeaf(executor_id_, PlanType::CTESource, schema_, fine_grained_shuffle, req_id) , sample_block(sample_block_) + , partition_col_ids(partition_col_ids_) + , partition_col_collators(partition_col_collators_) {} void finalizeImpl(const Names & parent_require) override; @@ -54,6 +59,8 @@ class PhysicalCTESource : public PhysicalLeaf private: Block sample_block; String query_id_and_cte_id; - UInt64 cte_id = 0; // TODO initialize it from tipb + UInt64 cte_id = 0; + std::vector partition_col_ids; + TiDB::TiDBCollators partition_col_collators; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp index c4e30572e43..469696af5b4 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp @@ -35,9 +35,11 @@ PhysicalPlanNodePtr PhysicalExchangeSender::build( { RUNTIME_CHECK(child); - std::vector partition_col_ids = ExchangeSenderInterpreterHelper::genPartitionColIds(exchange_sender); - TiDB::TiDBCollators partition_col_collators - = ExchangeSenderInterpreterHelper::genPartitionColCollators(exchange_sender); + std::vector partition_col_ids + = ExchangeSenderInterpreterHelper::genPartitionColIds(exchange_sender.partition_keys()); + TiDB::TiDBCollators partition_col_collators = ExchangeSenderInterpreterHelper::genPartitionColCollators( + exchange_sender.partition_keys(), + exchange_sender.types()); auto physical_exchange_sender = std::make_shared( executor_id, diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index e089ceb4312..8da819b34b9 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -30,11 +30,15 @@ class CTESourceOp : public SourceOp PipelineExecutorContext & exec_context_, const String & req_id, const String & query_id_and_cte_id_, - CTEManager * cte_manager_) + CTEManager * cte_manager_, + std::vector partition_col_ids_, + TiDB::TiDBCollators partition_col_collators_) : SourceOp(exec_context_, req_id) , query_id_and_cte_id(query_id_and_cte_id_) , cte_manager(cte_manager_) , cte(cte_manager_->getCTE(query_id_and_cte_id_)) + , partition_col_ids(partition_col_ids_) + , partition_col_collators(partition_col_collators_) {} ~CTESourceOp() override { assert(!this->cte); } @@ -56,5 +60,8 @@ class CTESourceOp : public SourceOp uint64_t total_rows{}; std::queue block_queue; + + std::vector partition_col_ids; + TiDB::TiDBCollators partition_col_collators; }; } // namespace DB From 6fb66a59b19a0fbb4c9f4ce506a78fe6240f3ca0 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 8 Apr 2025 22:21:39 +0800 Subject: [PATCH 017/118] tweaking --- dbms/src/Flash/Planner/PhysicalPlan.cpp | 30 +++++++++---------- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 5 +++- .../Flash/Planner/Plans/PhysicalCTESource.h | 4 ++- 3 files changed, 22 insertions(+), 17 deletions(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 48dc61191e2..7ccd95bc548 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -240,22 +240,22 @@ void PhysicalPlan::build(const tipb::Executor * executor) GET_METRIC(tiflash_coprocessor_executor_count, type_expand).Increment(); pushBack(PhysicalExpand2::build(context, executor_id, log, executor->expand2(), popBack())); break; - - { - // This is tmp code for tipb::ExecType::CTESource - auto fine_grained_shuffle = FineGrainedShuffle(executor); - GET_METRIC(tiflash_coprocessor_executor_count, type_cte_source).Increment(); - pushBack(PhysicalCTESource::build(context, executor_id, log, fine_grained_shuffle)); - } - { - // This is tmp code for tipb::ExecType::CTESink - auto fine_grained_shuffle = FineGrainedShuffle(executor); - GET_METRIC(tiflash_coprocessor_executor_count, type_cte_sink).Increment(); - pushBack(PhysicalCTESink::build(executor_id, log, fine_grained_shuffle, popBack())); - } } - // TODO add tipb::ExecType::CTESource - // TODO add tipb::ExecType::CTESink + case tipb::ExecType::TypeCTESource: + { + auto fine_grained_shuffle = FineGrainedShuffle(executor); + GET_METRIC(tiflash_coprocessor_executor_count, type_cte_source).Increment(); + pushBack(PhysicalCTESource::build(context, executor_id, log, fine_grained_shuffle, executor->cte_source())); + break; + } + case tipb::ExecType::TypeCTESink: + { + auto fine_grained_shuffle = FineGrainedShuffle(executor); + GET_METRIC(tiflash_coprocessor_executor_count, type_cte_sink).Increment(); + pushBack( + PhysicalCTESink::build(executor_id, log, fine_grained_shuffle, popBack(), executor->cte_sink().cte_id())); + break; + } default: throw TiFlashException( fmt::format("{} executor is not supported", fmt::underlying(executor->tp())), diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 7d716f2f76e..61aaf6f197b 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -49,7 +49,10 @@ PhysicalPlanNodePtr PhysicalCTESource::build( schema, fine_grained_shuffle, log->identifier(), - Block(schema)); + Block(schema), + cte_source.cte_id(), + partition_col_ids, + partition_col_collators); return physical_exchange_receiver; } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h index aa9f90d9728..484a8b13234 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -37,10 +37,12 @@ class PhysicalCTESource : public PhysicalLeaf const FineGrainedShuffle & fine_grained_shuffle, const String & req_id, const Block & sample_block_, + UInt64 cte_id_, const std::vector partition_col_ids_, const TiDB::TiDBCollators partition_col_collators_) : PhysicalLeaf(executor_id_, PlanType::CTESource, schema_, fine_grained_shuffle, req_id) , sample_block(sample_block_) + , cte_id(cte_id_) , partition_col_ids(partition_col_ids_) , partition_col_collators(partition_col_collators_) {} @@ -59,7 +61,7 @@ class PhysicalCTESource : public PhysicalLeaf private: Block sample_block; String query_id_and_cte_id; - UInt64 cte_id = 0; + UInt64 cte_id; std::vector partition_col_ids; TiDB::TiDBCollators partition_col_collators; }; From 3dcad01458b8f2a90f5ab8597162d893601f3ea6 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 9 Apr 2025 11:04:39 +0800 Subject: [PATCH 018/118] save --- .../ExchangeSenderInterpreterHelper.cpp | 22 +++++++++---------- .../ExchangeSenderInterpreterHelper.h | 8 +++---- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 16 +++----------- .../Flash/Planner/Plans/PhysicalCTESource.h | 6 +---- .../Planner/Plans/PhysicalExchangeSender.cpp | 8 +++---- dbms/src/Operators/CTESource.h | 9 +------- 6 files changed, 21 insertions(+), 48 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.cpp b/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.cpp index e5d51963e6a..0a858a785dd 100644 --- a/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.cpp +++ b/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.cpp @@ -23,10 +23,10 @@ namespace DB::ExchangeSenderInterpreterHelper { -std::vector genPartitionColIds(const ::google::protobuf::RepeatedPtrField<::tipb::Expr> & partition_keys) +std::vector genPartitionColIds(const tipb::ExchangeSender & exchange_sender) { std::vector partition_col_ids; - for (const auto & part_key : partition_keys) + for (const auto & part_key : exchange_sender.partition_keys()) { if (unlikely(!isColumnExpr(part_key))) { @@ -41,15 +41,13 @@ std::vector genPartitionColIds(const ::google::protobuf::RepeatedPtrField return partition_col_ids; } -TiDB::TiDBCollators genPartitionColCollators( - const ::google::protobuf::RepeatedPtrField<::tipb::Expr> & partition_keys, - const ::google::protobuf::RepeatedPtrField<::tipb::FieldType> & types) +TiDB::TiDBCollators genPartitionColCollators(const tipb::ExchangeSender & exchange_sender) { TiDB::TiDBCollators partition_col_collators; - auto type_num = types.size(); + const auto & part_keys = exchange_sender.partition_keys(); /// in case TiDB is an old version, it has no collation info - bool has_collator_info = type_num != 0; - if (unlikely(has_collator_info && partition_keys.size() != type_num)) + bool has_collator_info = exchange_sender.types_size() != 0; + if (unlikely(has_collator_info && part_keys.size() != exchange_sender.types_size())) { throw TiFlashException( fmt::format( @@ -58,12 +56,12 @@ TiDB::TiDBCollators genPartitionColCollators( __PRETTY_FUNCTION__), Errors::Coprocessor::BadRequest); } - for (int i = 0; i < partition_keys.size(); ++i) + for (int i = 0; i < part_keys.size(); ++i) { - const auto & expr = partition_keys[i]; + const auto & expr = part_keys[i]; if (has_collator_info && removeNullable(getDataTypeByFieldTypeForComputingLayer(expr.field_type()))->isString()) { - partition_col_collators.emplace_back(getCollatorFromFieldType(types.at(i))); + partition_col_collators.emplace_back(getCollatorFromFieldType(exchange_sender.types(i))); } else { @@ -72,4 +70,4 @@ TiDB::TiDBCollators genPartitionColCollators( } return partition_col_collators; } -} // namespace DB::ExchangeSenderInterpreterHelper +} // namespace DB::ExchangeSenderInterpreterHelper \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.h b/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.h index c1b88cf2d04..a7e980aac67 100644 --- a/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.h +++ b/dbms/src/Flash/Coprocessor/ExchangeSenderInterpreterHelper.h @@ -20,9 +20,7 @@ namespace DB::ExchangeSenderInterpreterHelper { -std::vector genPartitionColIds(const ::google::protobuf::RepeatedPtrField<::tipb::Expr> & partition_keys); +std::vector genPartitionColIds(const tipb::ExchangeSender & exchange_sender); -TiDB::TiDBCollators genPartitionColCollators( - const ::google::protobuf::RepeatedPtrField<::tipb::Expr> & partition_keys, - const ::google::protobuf::RepeatedPtrField<::tipb::FieldType> & types); -} // namespace DB::ExchangeSenderInterpreterHelper +TiDB::TiDBCollators genPartitionColCollators(const tipb::ExchangeSender & exchange_sender); +} // namespace DB::ExchangeSenderInterpreterHelper \ No newline at end of file diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 61aaf6f197b..255c598ba7c 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -30,11 +30,6 @@ PhysicalPlanNodePtr PhysicalCTESource::build( const FineGrainedShuffle & fine_grained_shuffle, const tipb::CTESource & cte_source) { - std::vector partition_col_ids - = ExchangeSenderInterpreterHelper::genPartitionColIds(cte_source.partition_keys()); - TiDB::TiDBCollators partition_col_collators - = ExchangeSenderInterpreterHelper::genPartitionColCollators(cte_source.partition_keys(), cte_source.types()); - DAGSchema dag_schema; for (int i = 0; i < cte_source.field_types_size(); ++i) { @@ -44,16 +39,13 @@ PhysicalPlanNodePtr PhysicalCTESource::build( } NamesAndTypes schema = toNamesAndTypes(dag_schema); - auto physical_exchange_receiver = std::make_shared( + return std::make_shared( executor_id, schema, fine_grained_shuffle, log->identifier(), Block(schema), - cte_source.cte_id(), - partition_col_ids, - partition_col_collators); - return physical_exchange_receiver; + cte_source.cte_id()); } void PhysicalCTESource::buildPipelineExecGroupImpl( @@ -74,9 +66,7 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( exec_context, log->identifier(), query_id_and_cte_id, - context.getCTEManager(), - this->partition_col_ids, - this->partition_col_collators)); + context.getCTEManager())); } context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h index 484a8b13234..493ab1bf46a 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -37,14 +37,10 @@ class PhysicalCTESource : public PhysicalLeaf const FineGrainedShuffle & fine_grained_shuffle, const String & req_id, const Block & sample_block_, - UInt64 cte_id_, - const std::vector partition_col_ids_, - const TiDB::TiDBCollators partition_col_collators_) + UInt64 cte_id_) : PhysicalLeaf(executor_id_, PlanType::CTESource, schema_, fine_grained_shuffle, req_id) , sample_block(sample_block_) , cte_id(cte_id_) - , partition_col_ids(partition_col_ids_) - , partition_col_collators(partition_col_collators_) {} void finalizeImpl(const Names & parent_require) override; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp index 469696af5b4..c4e30572e43 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp @@ -35,11 +35,9 @@ PhysicalPlanNodePtr PhysicalExchangeSender::build( { RUNTIME_CHECK(child); - std::vector partition_col_ids - = ExchangeSenderInterpreterHelper::genPartitionColIds(exchange_sender.partition_keys()); - TiDB::TiDBCollators partition_col_collators = ExchangeSenderInterpreterHelper::genPartitionColCollators( - exchange_sender.partition_keys(), - exchange_sender.types()); + std::vector partition_col_ids = ExchangeSenderInterpreterHelper::genPartitionColIds(exchange_sender); + TiDB::TiDBCollators partition_col_collators + = ExchangeSenderInterpreterHelper::genPartitionColCollators(exchange_sender); auto physical_exchange_sender = std::make_shared( executor_id, diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 8da819b34b9..e089ceb4312 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -30,15 +30,11 @@ class CTESourceOp : public SourceOp PipelineExecutorContext & exec_context_, const String & req_id, const String & query_id_and_cte_id_, - CTEManager * cte_manager_, - std::vector partition_col_ids_, - TiDB::TiDBCollators partition_col_collators_) + CTEManager * cte_manager_) : SourceOp(exec_context_, req_id) , query_id_and_cte_id(query_id_and_cte_id_) , cte_manager(cte_manager_) , cte(cte_manager_->getCTE(query_id_and_cte_id_)) - , partition_col_ids(partition_col_ids_) - , partition_col_collators(partition_col_collators_) {} ~CTESourceOp() override { assert(!this->cte); } @@ -60,8 +56,5 @@ class CTESourceOp : public SourceOp uint64_t total_rows{}; std::queue block_queue; - - std::vector partition_col_ids; - TiDB::TiDBCollators partition_col_collators; }; } // namespace DB From 3f78cfb07012481bfceed7cc9ae36978f829a867 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 16 Apr 2025 19:14:08 +0800 Subject: [PATCH 019/118] save --- dbms/src/Core/CTESpill.cpp | 20 ++++++++++++++++ dbms/src/Core/CTESpill.h | 44 ++++++++++++++++++++++++++++++++++ dbms/src/Operators/CTESinkOp.h | 1 + 3 files changed, 65 insertions(+) create mode 100644 dbms/src/Core/CTESpill.cpp create mode 100644 dbms/src/Core/CTESpill.h diff --git a/dbms/src/Core/CTESpill.cpp b/dbms/src/Core/CTESpill.cpp new file mode 100644 index 00000000000..95318e00d1f --- /dev/null +++ b/dbms/src/Core/CTESpill.cpp @@ -0,0 +1,20 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include + +namespace DB +{ + +} // namespace DB diff --git a/dbms/src/Core/CTESpill.h b/dbms/src/Core/CTESpill.h new file mode 100644 index 00000000000..419965e0328 --- /dev/null +++ b/dbms/src/Core/CTESpill.h @@ -0,0 +1,44 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include + +#include +#include +#include + +namespace DB +{ +// TODO we can use WriteBufferFromWritableFile(used SpillHandler::SpillWriter) to write file +// TODO we can use ReadBufferFromRandomAccessFile to write file +class CTESpill +{ +public: + void writeBlocks(Blocks && blocks); // TODO implement + void readBlockAt(Int64 idx) const; // TODO implement + Int64 blockNum() const; // TODO implement + +private: + std::shared_mutex rw_lock; + std::vector> block_offsets; + std::vector spilled_files; + SpillConfig config; // TODO initialize it + + // TODO maybe we need a queue to transfer block +}; +} // namespace DB diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 632815d6b7e..a84fa2a8ec4 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -17,6 +17,7 @@ #include #include #include +#include namespace DB { From bd30e212184adc94eb82786750c52f2de06c868b Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 18 Apr 2025 17:56:30 +0800 Subject: [PATCH 020/118] save --- dbms/src/Common/TiFlashMetrics.h | 4 +- dbms/src/Core/CTESpill.h | 7 +- .../Schedule/Tasks/PipeConditionVariable.h | 6 ++ dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 2 + dbms/src/Operators/CTE.cpp | 98 ++++++++++++++----- dbms/src/Operators/CTE.h | 38 +++++-- dbms/src/Operators/CTESinkOp.cpp | 24 ++++- dbms/src/Operators/CTESinkOp.h | 2 + dbms/src/Operators/CTESource.h | 2 +- 9 files changed, 141 insertions(+), 42 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index bdedf40d6b7..1ebc249e914 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -769,7 +769,9 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_wait_on_tunnel_sender_write, {"type", "wait_on_tunnel_sender_write"}), \ F(type_wait_on_join_build, {"type", "wait_on_join_build"}), \ F(type_wait_on_join_probe, {"type", "wait_on_join_probe"}), \ - F(type_wait_on_result_queue_write, {"type", "wait_on_result_queue_write"})) \ + F(type_wait_on_result_queue_write, {"type", "wait_on_result_queue_write"}), \ + F(type_wait_on_cte_sink, {"type", "wait_on_cte_sink"}), \ + F(type_wait_on_cte_source, {"type", "wait_on_cte_source"})) \ M(tiflash_pipeline_task_duration_seconds, \ "Bucketed histogram of pipeline task duration in seconds", \ Histogram, /* these command usually cost several hundred milliseconds to several seconds, increase the start bucket to 5ms */ \ diff --git a/dbms/src/Core/CTESpill.h b/dbms/src/Core/CTESpill.h index 419965e0328..9c2f2ad2d8c 100644 --- a/dbms/src/Core/CTESpill.h +++ b/dbms/src/Core/CTESpill.h @@ -29,8 +29,11 @@ namespace DB class CTESpill { public: - void writeBlocks(Blocks && blocks); // TODO implement - void readBlockAt(Int64 idx) const; // TODO implement + // TODO maybe we need an initialization function as spill may not be triggered and we can initialize it until spill is triggered + + // TODO all function need lock as CTESpill may be concurrently accessed + void writeBlocks(const Blocks & blocks); // TODO implement, return value need other types + Block readBlockAt(Int64 idx) const; // TODO implement Int64 blockNum() const; // TODO implement private: diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index b24bb129edc..6393d53ad5a 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -57,6 +57,12 @@ class PipeConditionVariable case NotifyType::WAIT_ON_SHARED_QUEUE_WRITE: GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_shared_queue_write).Increment(change); break; + case NotifyType::WAIT_ON_CTE_SINK: + GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte_sink); + break; + case NotifyType::WAIT_ON_CTE_SOURCE: + GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte_source); + break; } } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h index 46670798b5c..6f05e794f1e 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -57,6 +57,8 @@ enum class NotifyType WAIT_ON_JOIN_BUILD_FINISH, WAIT_ON_JOIN_PROBE_FINISH, WAIT_ON_RESULT_QUEUE_WRITE, + WAIT_ON_CTE_SINK, + WAIT_ON_CTE_SOURCE, }; class PipelineExecutorContext; diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 3569013e8a7..c4af7079e1e 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -17,45 +17,82 @@ #include #include #include +#include "Common/Exception.h" +#include "Core/Block.h" namespace DB { -FetchStatus CTE::checkAvailableBlockAt(size_t idx) +std::pair CTE::tryGetBlockAt(size_t idx) { - std::shared_lock lock(this->rw_lock); + std::shared_lock lock(this->rw_lock, std::defer_lock); + { + std::shared_lock status_lock(this->aux_rw_lock); + if (this->cte_status != CTE::Normal) + return {Status::IOOut, Block()}; + + // This function is called in cpu pool, we don't want to wait for this lock too long. + // This lock may be held when spill is in execution. So we need ensure that cte status is not changed + lock.lock(); + } + if (this->is_spill_triggered) + { + auto spilled_block_num = static_cast(this->cte_spill.blockNum()); + if (idx < spilled_block_num) + return {Status::IOIn, Block()}; + + idx -= spilled_block_num; + } + auto block_num = this->blocks.size(); if (block_num <= idx) { if (this->is_eof) - return FetchStatus::Eof; + return {Status::Eof, Block()}; else - return FetchStatus::Waiting; + return {Status::Waiting, Block()}; } - return FetchStatus::Ok; + return {Status::Ok, this->blocks[idx]}; } -std::pair CTE::tryGetBlockAt(size_t idx) +std::pair CTE::getBlockFromDisk(size_t idx) { std::shared_lock lock(this->rw_lock); - auto block_num = this->blocks.size(); - if (block_num <= idx) - { - if (this->is_eof) - return {FetchStatus::Eof, Block()}; - else - return {FetchStatus::Waiting, Block()}; - } - return {FetchStatus::Ok, this->blocks[idx]}; + if unlikely (!this->is_spill_triggered) + // We can call this function only when spill is triggered + throw Exception("Spill should be triggered"); + + if unlikely (static_cast(this->cte_spill.blockNum()) <= idx) + throw Exception("Requested block is not in disk"); + + return {Status::Ok, this->cte_spill.readBlockAt(idx)}; } -void CTE::pushBlock(const Block & block) +Status CTE::pushBlock(const Block & block) { - std::unique_lock lock(this->rw_lock); + std::unique_lock lock(this->rw_lock, std::defer_lock); + Status ret = Status::Ok; + { + std::unique_lock status_lock(this->aux_rw_lock); + if (this->cte_status != CTE::Normal) + return Status::IOOut; + + + // This function is called in cpu pool, we don't want to wait for this lock too long. + // This lock may be held when spill is in execution. So we need ensure that cte status is not changed + lock.lock(); + + this->memory_usage += block.bytes(); + if (this->memory_usage >= this->memory_threshold) + { + this->cte_status = CTE::NeedSpill; + ret = Status::IOOut; + } + } - this->memory_usage += block.bytes(); if unlikely (this->blocks.empty()) this->pipe_cv.notifyAll(); this->blocks.push_back(block); + return ret; } void CTE::notifyEOF() @@ -67,16 +104,23 @@ void CTE::notifyEOF() this->pipe_cv.notifyAll(); } +void CTE::spillBlocks() +{ + std::unique_lock lock(this->rw_lock); + + this->cte_spill.writeBlocks(this->blocks); // TODO need to handle return value + this->blocks.resize(0); + this->memory_usage = 0; + + // TODO handle tmp_blocks + + // Many tasks may be waiting for the finish of spill + this->pipe_cv.notifyAll(); +} + void CTE::registerTask(TaskPtr && task) { - { - std::unique_lock lock(this->rw_lock); - if (!this->hasDataNoLock()) - { - pipe_cv.registerTask(std::move(task)); - return; - } - } - this->pipe_cv.notifyTaskDirectly(std::move(task)); + // TODO can we directly register the task? Can we ensure that someone must wake it up? + pipe_cv.registerTask(std::move(task)); } } // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 18d05626958..ea5bc7c870c 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -16,18 +16,22 @@ #include #include +#include #include #include +#include #include #include namespace DB { -enum class FetchStatus +enum class Status { Ok, Waiting, + IOOut, + IOIn, Eof, Cancelled }; @@ -37,26 +41,42 @@ class CTE : public NotifyFuture public: ~CTE() override = default; - std::pair tryGetBlockAt(size_t idx); - FetchStatus checkAvailableBlockAt(size_t idx); - void pushBlock(const Block & block); + enum CTEStatus + { + Normal = 0, + NeedSpill, + InSpilling, + }; + + std::pair tryGetBlockAt(size_t idx); + std::pair getBlockFromDisk(size_t idx); + Status pushBlock(const Block & block); void notifyEOF(); + // TODO should have return value to indicate if spill successes + void spillBlocks(); + void registerTask(TaskPtr && task) override; private: - // Return true if CTE has data - inline bool hasDataNoLock() const { return !this->blocks.empty(); } - std::shared_mutex rw_lock; Blocks blocks; size_t memory_usage = 0; + size_t memory_threshold = 0; // TODO initialize it - // Tasks in WAITING_FOR_NOTIFY are saved in this deque - std::deque waiting_tasks; + std::atomic_int8_t cte_status; // TODO initialize it + std::shared_mutex aux_rw_lock; + + // TODO handle this, some blocks can not be spilled when spill is in execution, they can only be stored temporary + Blocks tmp_blocks; + + // TODO tasks waiting for notify may be lack of data or waiting for spill, do we need two pipe_cv to handle this? PipeConditionVariable pipe_cv; + CTESpill cte_spill; + bool is_eof = false; + bool is_spill_triggered = false; }; } // namespace DB diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 39eebf16917..008d0d2492e 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB { @@ -36,7 +37,26 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) return OperatorStatus::FINISHED; } this->total_rows += block.rows(); - this->cte->pushBlock(block); - return OperatorStatus::NEED_INPUT; + auto status = this->cte->pushBlock(block); + switch (status) + { + case Status::IOOut: + return OperatorStatus::IO_OUT; + case Status::Ok: + return OperatorStatus::NEED_INPUT; + default: + throw Exception(fmt::format("Get unexpected Status: {}", magic_enum::enum_name(status))); + } +} + +OperatorStatus CTESinkOp::executeIOImpl() +{ + // TODO handle return value + this->cte->spillBlocks(); +} + +OperatorStatus CTESinkOp::awaitImpl() +{ + } } // namespace DB diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index a84fa2a8ec4..fbbe8a5dc76 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -43,6 +43,8 @@ class CTESinkOp : public SinkOp protected: void operateSuffixImpl() override; OperatorStatus writeImpl(Block && block) override; + OperatorStatus executeIOImpl() override; + OperatorStatus awaitImpl() override; private: String query_id_and_cte_id; diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index e089ceb4312..22ea2b76476 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -46,7 +46,7 @@ class CTESourceOp : public SourceOp OperatorStatus readImpl(Block & block) override; - OperatorStatus awaitImpl() override; + OperatorStatus awaitImpl() override; // TODO implement awaitImpl private: String query_id_and_cte_id; From f3649f04bedcad692e439f7f5c03166b6eaf14d3 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 21 Apr 2025 09:45:54 +0800 Subject: [PATCH 021/118] tweaking --- dbms/src/Operators/CTE.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index c4af7079e1e..0fa63e4519e 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -58,7 +58,7 @@ std::pair CTE::getBlockFromDisk(size_t idx) { std::shared_lock lock(this->rw_lock); if unlikely (!this->is_spill_triggered) - // We can call this function only when spill is triggered + // We can call this function only when spill is triggered throw Exception("Spill should be triggered"); if unlikely (static_cast(this->cte_spill.blockNum()) <= idx) From 482daecd3af93d3099e22cea59dbed9896b320ef Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 21 Apr 2025 09:46:08 +0800 Subject: [PATCH 022/118] save --- dbms/src/Operators/CTE.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 0fa63e4519e..c4af7079e1e 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -58,7 +58,7 @@ std::pair CTE::getBlockFromDisk(size_t idx) { std::shared_lock lock(this->rw_lock); if unlikely (!this->is_spill_triggered) - // We can call this function only when spill is triggered + // We can call this function only when spill is triggered throw Exception("Spill should be triggered"); if unlikely (static_cast(this->cte_spill.blockNum()) <= idx) From 6671078db00378445b19c83c7f89e8cdb3081360 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 21 Apr 2025 10:05:57 +0800 Subject: [PATCH 023/118] save --- dbms/src/Operators/CTE.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index c4af7079e1e..0fa63e4519e 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -58,7 +58,7 @@ std::pair CTE::getBlockFromDisk(size_t idx) { std::shared_lock lock(this->rw_lock); if unlikely (!this->is_spill_triggered) - // We can call this function only when spill is triggered + // We can call this function only when spill is triggered throw Exception("Spill should be triggered"); if unlikely (static_cast(this->cte_spill.blockNum()) <= idx) From 3610df7c1525564301e7c61d69333a7e5505da44 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 21 Apr 2025 10:06:18 +0800 Subject: [PATCH 024/118] save --- dbms/src/Operators/CTE.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 0fa63e4519e..c4af7079e1e 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -58,7 +58,7 @@ std::pair CTE::getBlockFromDisk(size_t idx) { std::shared_lock lock(this->rw_lock); if unlikely (!this->is_spill_triggered) - // We can call this function only when spill is triggered + // We can call this function only when spill is triggered throw Exception("Spill should be triggered"); if unlikely (static_cast(this->cte_spill.blockNum()) <= idx) From 97e4611b30b77117d2df8f4c45b8106dcab14a92 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 24 Apr 2025 16:15:11 +0800 Subject: [PATCH 025/118] save --- dbms/src/Operators/CTE.cpp | 31 ++++++++++++++++++++++---- dbms/src/Operators/CTE.h | 7 ++++-- dbms/src/Operators/CTESinkOp.cpp | 6 ++++- dbms/src/Operators/CTESource.cpp | 38 +++++++++++++++++++------------- dbms/src/Operators/CTESource.h | 8 +++++++ 5 files changed, 68 insertions(+), 22 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index c4af7079e1e..e93d861cd07 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -74,7 +74,11 @@ Status CTE::pushBlock(const Block & block) { std::unique_lock status_lock(this->aux_rw_lock); if (this->cte_status != CTE::Normal) + { + // Block memory usage will be calculated after the finish of spill + this->tmp_blocks.push_back(block); return Status::IOOut; + } // This function is called in cpu pool, we don't want to wait for this lock too long. @@ -108,11 +112,24 @@ void CTE::spillBlocks() { std::unique_lock lock(this->rw_lock); - this->cte_spill.writeBlocks(this->blocks); // TODO need to handle return value - this->blocks.resize(0); - this->memory_usage = 0; + while (true) + { + this->cte_spill.writeBlocks(this->blocks); // TODO need to handle return value + this->blocks.clear(); + this->memory_usage = 0; + + std::unique_lock aux_lock(this->aux_rw_lock); + for (const auto & block : this->tmp_blocks) + { + this->blocks.push_back(block); + this->memory_usage += block.bytes(); + } + + this->tmp_blocks.clear(); - // TODO handle tmp_blocks + if (this->memory_usage < this->memory_threshold) + break; + } // Many tasks may be waiting for the finish of spill this->pipe_cv.notifyAll(); @@ -123,4 +140,10 @@ void CTE::registerTask(TaskPtr && task) // TODO can we directly register the task? Can we ensure that someone must wake it up? pipe_cv.registerTask(std::move(task)); } + +CTE::CTEStatus CTE::getStatus() +{ + std::shared_lock lock(this->aux_rw_lock); + return this->cte_status; +} } // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index ea5bc7c870c..30157cca78b 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -50,6 +50,7 @@ class CTE : public NotifyFuture std::pair tryGetBlockAt(size_t idx); std::pair getBlockFromDisk(size_t idx); + CTEStatus getStatus(); Status pushBlock(const Block & block); void notifyEOF(); @@ -65,12 +66,14 @@ class CTE : public NotifyFuture size_t memory_usage = 0; size_t memory_threshold = 0; // TODO initialize it - std::atomic_int8_t cte_status; // TODO initialize it - std::shared_mutex aux_rw_lock; + CTEStatus cte_status = CTEStatus::Normal; // TODO handle this, some blocks can not be spilled when spill is in execution, they can only be stored temporary Blocks tmp_blocks; + // Protecting cte_status and tmp_blocks + std::shared_mutex aux_rw_lock; + // TODO tasks waiting for notify may be lack of data or waiting for spill, do we need two pipe_cv to handle this? PipeConditionVariable pipe_cv; diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 008d0d2492e..98bda173701 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -53,10 +53,14 @@ OperatorStatus CTESinkOp::executeIOImpl() { // TODO handle return value this->cte->spillBlocks(); + return OperatorStatus::NEED_INPUT; // TODO handle } OperatorStatus CTESinkOp::awaitImpl() { - + // CTESinkOp waits for the finish of spill + if (this->cte->getStatus() == CTE::CTEStatus::Normal) + return OperatorStatus::NEED_INPUT; + return OperatorStatus::WAITING; } } // namespace DB diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index 6f20a238e08..489d034c4ae 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -25,42 +25,50 @@ void CTESourceOp::operateSuffixImpl() LOG_DEBUG(log, "finish read {} rows from cte source", total_rows); } -// TODO in some cases, source needs to manually filter some data when cte saves all data(start here) OperatorStatus CTESourceOp::readImpl(Block & block) { auto res = this->cte->tryGetBlockAt(this->block_fetch_idx); switch (res.first) { - case DB::FetchStatus::Eof: - case DB::FetchStatus::Ok: + case Status::Eof: + case Status::Ok: block = res.second; ++(this->block_fetch_idx); return OperatorStatus::HAS_OUTPUT; - case DB::FetchStatus::Waiting: + case Status::IOIn: + // Expected block is in disk, we need to read it from disk + return OperatorStatus::IO_IN; + case Status::IOOut: + { + // CTE is spilling blocks to disk, we need to wait the finish of spill + this->wait_type = CTESourceOp::Spill; + return OperatorStatus::WAITING; + } + case Status::Waiting: if unlikely (this->block_fetch_idx == 0) // CTE has not begun to receive data yet when block_fetch_idx == 0 // So we need to wait the notify from CTE return OperatorStatus::WAIT_FOR_NOTIFY; else + { + // CTE not have enough block, we need to wait for it + this->wait_type = CTESourceOp::NeedMoreBlock; return OperatorStatus::WAITING; - case DB::FetchStatus::Cancelled: + } + case Status::Cancelled: return OperatorStatus::CANCELLED; } } OperatorStatus CTESourceOp::awaitImpl() { - auto res = this->cte->checkAvailableBlockAt(this->block_fetch_idx); - switch (res) + if (this->wait_type == CTESourceOp::NeedMoreBlock) { - case DB::FetchStatus::Eof: - case DB::FetchStatus::Ok: - // Do not add block_fetch_idx here, as we just judge if there are available blocks - return OperatorStatus::HAS_OUTPUT; - case DB::FetchStatus::Waiting: - return OperatorStatus::WAITING; - case DB::FetchStatus::Cancelled: - return OperatorStatus::CANCELLED; + + } + else + { + } } } // namespace DB diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 22ea2b76476..614213e19a3 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -35,6 +35,7 @@ class CTESourceOp : public SourceOp , query_id_and_cte_id(query_id_and_cte_id_) , cte_manager(cte_manager_) , cte(cte_manager_->getCTE(query_id_and_cte_id_)) + , wait_type(NeedMoreBlock) {} ~CTESourceOp() override { assert(!this->cte); } @@ -49,6 +50,12 @@ class CTESourceOp : public SourceOp OperatorStatus awaitImpl() override; // TODO implement awaitImpl private: + enum WaitType + { + NeedMoreBlock, + Spill, + }; + String query_id_and_cte_id; CTEManager * cte_manager; std::shared_ptr cte; @@ -56,5 +63,6 @@ class CTESourceOp : public SourceOp uint64_t total_rows{}; std::queue block_queue; + WaitType wait_type; }; } // namespace DB From 75e3855a9933768ff0267bba9e561d2a9d27692d Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 6 May 2025 15:21:15 +0800 Subject: [PATCH 026/118] save --- dbms/src/Operators/CTE.cpp | 10 +++---- dbms/src/Operators/CTE.h | 5 ++-- dbms/src/Operators/CTESinkOp.h | 2 +- dbms/src/Operators/CTESource.cpp | 47 +++++++++++++++++++++++++++++--- dbms/src/Operators/CTESource.h | 7 +++-- 5 files changed, 55 insertions(+), 16 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index e93d861cd07..023ee877f53 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -13,12 +13,12 @@ // limitations under the License. #include +#include +#include #include #include #include -#include "Common/Exception.h" -#include "Core/Block.h" namespace DB { @@ -49,12 +49,12 @@ std::pair CTE::tryGetBlockAt(size_t idx) if (this->is_eof) return {Status::Eof, Block()}; else - return {Status::Waiting, Block()}; + return {Status::BlockUnavailable, Block()}; } return {Status::Ok, this->blocks[idx]}; } -std::pair CTE::getBlockFromDisk(size_t idx) +Block CTE::getBlockFromDisk(size_t idx) { std::shared_lock lock(this->rw_lock); if unlikely (!this->is_spill_triggered) @@ -64,7 +64,7 @@ std::pair CTE::getBlockFromDisk(size_t idx) if unlikely (static_cast(this->cte_spill.blockNum()) <= idx) throw Exception("Requested block is not in disk"); - return {Status::Ok, this->cte_spill.readBlockAt(idx)}; + return this->cte_spill.readBlockAt(idx); } Status CTE::pushBlock(const Block & block) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 30157cca78b..1663d6d1374 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -20,7 +20,6 @@ #include #include -#include #include #include @@ -29,7 +28,7 @@ namespace DB enum class Status { Ok, - Waiting, + BlockUnavailable, // It means that we do not have specified block so far IOOut, IOIn, Eof, @@ -49,7 +48,7 @@ class CTE : public NotifyFuture }; std::pair tryGetBlockAt(size_t idx); - std::pair getBlockFromDisk(size_t idx); + Block getBlockFromDisk(size_t idx); CTEStatus getStatus(); Status pushBlock(const Block & block); void notifyEOF(); diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index fbbe8a5dc76..bbea134d4a9 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -43,7 +43,7 @@ class CTESinkOp : public SinkOp protected: void operateSuffixImpl() override; OperatorStatus writeImpl(Block && block) override; - OperatorStatus executeIOImpl() override; + OperatorStatus executeIOImpl() override; // TODO implement it OperatorStatus awaitImpl() override; private: diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index 489d034c4ae..1e082e50a4c 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -27,6 +27,13 @@ void CTESourceOp::operateSuffixImpl() OperatorStatus CTESourceOp::readImpl(Block & block) { + if (this->block_from_disk) + { + block = this->block_from_disk; + this->block_from_disk = Block(); + return OperatorStatus::HAS_OUTPUT; + } + auto res = this->cte->tryGetBlockAt(this->block_fetch_idx); switch (res.first) { @@ -44,7 +51,7 @@ OperatorStatus CTESourceOp::readImpl(Block & block) this->wait_type = CTESourceOp::Spill; return OperatorStatus::WAITING; } - case Status::Waiting: + case Status::BlockUnavailable: if unlikely (this->block_fetch_idx == 0) // CTE has not begun to receive data yet when block_fetch_idx == 0 // So we need to wait the notify from CTE @@ -60,15 +67,47 @@ OperatorStatus CTESourceOp::readImpl(Block & block) } } +OperatorStatus CTESourceOp::executeIOImpl() +{ + this->block_from_disk = this->cte->getBlockFromDisk(this->block_fetch_idx); + this->block_fetch_idx++; + return OperatorStatus::HAS_OUTPUT; +} + OperatorStatus CTESourceOp::awaitImpl() { - if (this->wait_type == CTESourceOp::NeedMoreBlock) + if (this->wait_type == CTESourceOp::WaitType::NeedMoreBlock) { - + auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx); + switch (ret.first) + { + case Status::IOOut: + this->wait_type = CTESourceOp::WaitType::Spill; + case Status::BlockUnavailable: + return OperatorStatus::WAITING; + case Status::Ok: + case Status::Eof: + return OperatorStatus::HAS_OUTPUT; + case Status::Cancelled: + return OperatorStatus::CANCELLED; + case Status::IOIn: + return OperatorStatus::IO_IN; + } + } + else if (this->wait_type == CTESourceOp::WaitType::Spill) + { + switch (this->cte->getStatus()) + { + case CTE::CTEStatus::Normal: + return OperatorStatus::HAS_OUTPUT; + case CTE::CTEStatus::NeedSpill: + case CTE::CTEStatus::InSpilling: + return OperatorStatus::WAITING; + } } else { - + throw Exception("Unexpected wait type"); } } } // namespace DB diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 614213e19a3..35f90f433b8 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -46,8 +46,8 @@ class CTESourceOp : public SourceOp void operateSuffixImpl() override; OperatorStatus readImpl(Block & block) override; - - OperatorStatus awaitImpl() override; // TODO implement awaitImpl + OperatorStatus executeIOImpl() override; + OperatorStatus awaitImpl() override; private: enum WaitType @@ -61,8 +61,9 @@ class CTESourceOp : public SourceOp std::shared_ptr cte; size_t block_fetch_idx = 0; + Block block_from_disk; + uint64_t total_rows{}; - std::queue block_queue; WaitType wait_type; }; } // namespace DB From 4515c3b7fb17b053ee93ddc5ddd6b8d876cc3cac Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 8 May 2025 09:44:52 +0800 Subject: [PATCH 027/118] save --- dbms/src/Core/CTESpill.cpp | 65 +++++++++++++++++++ dbms/src/Core/CTESpill.h | 23 +++++-- dbms/src/DataStreams/NativeBlockInputStream.h | 6 ++ .../DataStreams/NativeBlockOutputStream.cpp | 7 ++ .../src/DataStreams/NativeBlockOutputStream.h | 1 + dbms/src/IO/Buffer/ReadBuffer.h | 3 + dbms/src/Operators/CTE.cpp | 2 +- dbms/src/Operators/CTESinkOp.cpp | 3 +- dbms/src/Operators/CTESinkOp.h | 2 +- 9 files changed, 101 insertions(+), 11 deletions(-) diff --git a/dbms/src/Core/CTESpill.cpp b/dbms/src/Core/CTESpill.cpp index 95318e00d1f..4903af267d2 100644 --- a/dbms/src/Core/CTESpill.cpp +++ b/dbms/src/Core/CTESpill.cpp @@ -13,8 +13,73 @@ // limitations under the License. #include +#include + +#include +#include +#include namespace DB { +void CTESpill::writeBlocks(const Blocks & blocks) +{ + std::unique_lock lock(this->rw_lock); + + for (const auto & block : blocks) + { + if unlikely (this->spilled_files.back().isFull() || this->spilled_files.empty()) + { + // TODO create new spilled_files and write_read_files + } + + Int64 prev_block_offset; + Int64 current_block_file_idx = this->spilled_files.size() - 1; + if unlikely (this->block_offsets.empty() || this->block_offsets.back().first != current_block_file_idx) + prev_block_offset = 0; + else + prev_block_offset = this->block_offsets.back().second; + + const auto block_size = this->write_streams.back().writeAndReturnBlockSize(block); + this->block_offsets.push_back((std::make_pair(this->spilled_files.size()-1, prev_block_offset+block_size))); + } +} + +Block CTESpill::readBlockAt(Int64 idx) +{ + std::shared_lock lock(this->rw_lock); + if unlikely (idx >= static_cast(this->block_offsets.size())) + throw Exception(fmt::format("Requested block idx({}) is larger than total block number({})", idx, this->block_offsets.size())); + + auto block_location = this->block_offsets[idx]; + Int64 stream_idx = block_location.first; + Int64 block_offset = block_location.second; + Int64 block_size = this->getBlockSizeNoLock(idx); + + if (this->buf.size() < static_cast(block_size)) + this->buf.resize(block_size); + + this->read_streams[stream_idx].seek(block_offset); + return this->read_streams[stream_idx].read(); +} + +Int64 CTESpill::blockNum() +{ + std::shared_lock lock(this->rw_lock); + return this->block_offsets.size(); +} +Int64 CTESpill::getBlockSizeNoLock(Int64 idx) const +{ + if unlikely (idx >= static_cast(this->block_offsets.size())) + throw Exception(fmt::format("Requested block idx({}) is larger than total block number({})", idx, this->block_offsets.size())); + + if unlikely (idx == 0) + return this->block_offsets[0].second; + + Int64 prev_block_file_idx = this->block_offsets[idx-1].first; + if unlikely (prev_block_file_idx != this->block_offsets[idx].first) + return this->block_offsets[idx].second; + + return this->block_offsets[idx].second - this->block_offsets[idx-1].second; +} } // namespace DB diff --git a/dbms/src/Core/CTESpill.h b/dbms/src/Core/CTESpill.h index 9c2f2ad2d8c..36bb8040a0b 100644 --- a/dbms/src/Core/CTESpill.h +++ b/dbms/src/Core/CTESpill.h @@ -17,6 +17,9 @@ #include #include #include +#include +#include +#include #include #include @@ -24,8 +27,6 @@ namespace DB { -// TODO we can use WriteBufferFromWritableFile(used SpillHandler::SpillWriter) to write file -// TODO we can use ReadBufferFromRandomAccessFile to write file class CTESpill { public: @@ -33,15 +34,23 @@ class CTESpill // TODO all function need lock as CTESpill may be concurrently accessed void writeBlocks(const Blocks & blocks); // TODO implement, return value need other types - Block readBlockAt(Int64 idx) const; // TODO implement - Int64 blockNum() const; // TODO implement + Block readBlockAt(Int64 idx); + Int64 blockNum(); private: + Int64 getBlockSizeNoLock(Int64 idx) const; + std::shared_mutex rw_lock; - std::vector> block_offsets; std::vector spilled_files; + // TODO maybe we need to use compressed stream + // TODO maybe each one stream need one lock + std::vector read_streams; + std::vector write_streams; + std::vector> block_offsets; + FileProviderPtr file_provider; + Int64 write_offset; // Always write to the last file, so we need only one offset SpillConfig config; // TODO initialize it - - // TODO maybe we need a queue to transfer block + + std::vector buf; }; } // namespace DB diff --git a/dbms/src/DataStreams/NativeBlockInputStream.h b/dbms/src/DataStreams/NativeBlockInputStream.h index c08467a8a5a..f5bc84cb39a 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.h +++ b/dbms/src/DataStreams/NativeBlockInputStream.h @@ -101,6 +101,12 @@ class NativeBlockInputStream : public IProfilingBlockInputStream Block getHeader() const override; + void seek(Int64 pos) + { + this->istr.seek(pos); + this->istr.next(); // Trigger update for read buffer + } + protected: Block readImpl() override; diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index 4c981c06336..98a9d08cc7e 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -141,4 +141,11 @@ void NativeBlockOutputStream::write(const Block & block) } } +Int64 NativeBlockOutputStream::writeAndReturnBlockSize(const Block & block) +{ + Int64 prev_byte_count = this->ostr.count(); + this->write(block); + return this->ostr.count() - prev_byte_count; +} + } // namespace DB diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.h b/dbms/src/DataStreams/NativeBlockOutputStream.h index 9aa04e3c3b4..0fdc9b3bff2 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.h +++ b/dbms/src/DataStreams/NativeBlockOutputStream.h @@ -44,6 +44,7 @@ class NativeBlockOutputStream : public IBlockOutputStream Block getHeader() const override { return header; } void write(const Block & block) override; + Int64 writeAndReturnBlockSize(const Block & block); void flush() override; static void writeData( diff --git a/dbms/src/IO/Buffer/ReadBuffer.h b/dbms/src/IO/Buffer/ReadBuffer.h index 921d15eb90b..056b1a999a8 100644 --- a/dbms/src/IO/Buffer/ReadBuffer.h +++ b/dbms/src/IO/Buffer/ReadBuffer.h @@ -175,6 +175,9 @@ class ReadBuffer : public BufferBase */ virtual size_t readBig(char * to, size_t n) { return read(to, n); } + // Set the read position for file + virtual void seek(Int64) { throw Exception("Not implemented yet"); } + protected: /// The number of bytes to ignore from the initial position of `working_buffer` buffer. size_t working_buffer_offset = 0; diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 023ee877f53..97446a7c217 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -114,7 +114,7 @@ void CTE::spillBlocks() while (true) { - this->cte_spill.writeBlocks(this->blocks); // TODO need to handle return value + this->cte_spill.writeBlocks(this->blocks); this->blocks.clear(); this->memory_usage = 0; diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 98bda173701..003e3d4bb42 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -51,9 +51,8 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) OperatorStatus CTESinkOp::executeIOImpl() { - // TODO handle return value this->cte->spillBlocks(); - return OperatorStatus::NEED_INPUT; // TODO handle + return OperatorStatus::NEED_INPUT; } OperatorStatus CTESinkOp::awaitImpl() diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index bbea134d4a9..fbbe8a5dc76 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -43,7 +43,7 @@ class CTESinkOp : public SinkOp protected: void operateSuffixImpl() override; OperatorStatus writeImpl(Block && block) override; - OperatorStatus executeIOImpl() override; // TODO implement it + OperatorStatus executeIOImpl() override; OperatorStatus awaitImpl() override; private: From 25a9cf5d60aa909de85dfe0c37ef4e1cb8cf7cfc Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 8 May 2025 23:33:18 +0800 Subject: [PATCH 028/118] fix bugs --- dbms/src/Flash/Mpp/MPPTask.cpp | 15 ++++++++++-- dbms/src/Flash/Mpp/MPPTaskStatistics.cpp | 4 ++-- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 3 ++- .../src/Flash/Statistics/CommonExecutorImpl.h | 24 +++++++++++++++++++ .../ExecutorStatisticsCollector.cpp | 4 +++- .../Flash/Statistics/traverseExecutors.cpp | 5 ++++ dbms/src/Operators/CTESinkOp.cpp | 5 ---- dbms/src/Operators/CTESinkOp.h | 9 ++++++- dbms/src/Operators/CTESource.cpp | 6 ++--- dbms/src/Operators/CTESource.h | 15 +++++++++--- 10 files changed, 72 insertions(+), 18 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 5d500e994c1..c20ccaee0c8 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -205,6 +205,14 @@ void MPPTask::run() void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) { + if unlikely (!dag_context->dag_request.rootExecutor().has_exchange_sender()) + { + if unlikely (!dag_context->dag_request.rootExecutor().has_cte_sink()) + throw Exception("Task has either exchange sender or cte sink"); + + // There is no need to register tunnel for cte sink + return; + } auto tunnel_set_local = std::make_shared(log->identifier()); std::chrono::seconds timeout(task_request.timeout()); const auto & exchange_sender = dag_context->dag_request.rootExecutor().exchange_sender(); @@ -808,13 +816,16 @@ int MPPTask::estimateCountOfNewThreads() { auto query_executor = query_executor_holder.tryGet(); RUNTIME_CHECK_MSG( - query_executor && dag_context->tunnel_set != nullptr, + query_executor && (dag_context->tunnel_set != nullptr || dag_context->dag_request->root_executor().has_cte_sink()), "It should not estimate the threads for the uninitialized task {}", id.toString()); // Estimated count of new threads from query executor, MppTunnels, mpp_receivers. assert(query_executor.value()); - return (*query_executor)->estimateNewThreadCount() + 1 + dag_context->tunnel_set->getExternalThreadCnt() + int external_thread_count = 0; + if likely (dag_context->tunnel_set != nullptr) + external_thread_count = dag_context->tunnel_set->getExternalThreadCnt(); + return (*query_executor)->estimateNewThreadCount() + 1 + external_thread_count + new_thread_count_of_mpp_receiver; } diff --git a/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp b/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp index 0be18a3abff..8674cf0dcb1 100644 --- a/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp @@ -70,9 +70,9 @@ void MPPTaskStatistics::initializeExecutorDAG(DAGContext * dag_context_) assert(dag_context_->isMPPTask()); dag_context = dag_context_; const auto & root_executor = dag_context->dag_request.rootExecutor(); - if unlikely (!root_executor.has_exchange_sender()) + if unlikely (!root_executor.has_exchange_sender() && !root_executor.has_cte_sink()) throw TiFlashException( - "The root executor isn't ExchangeSender in MPP, which is unexpected.", + "The root executor isn't ExchangeSender or CTESink in MPP, which is unexpected.", Errors::Coprocessor::BadRequest); is_root = dag_context->isRootMPPTask(); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 255c598ba7c..6ae77656392 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -66,7 +66,8 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( exec_context, log->identifier(), query_id_and_cte_id, - context.getCTEManager())); + context.getCTEManager(), + schema)); } context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } diff --git a/dbms/src/Flash/Statistics/CommonExecutorImpl.h b/dbms/src/Flash/Statistics/CommonExecutorImpl.h index f37a0dea8cc..91adf116a92 100644 --- a/dbms/src/Flash/Statistics/CommonExecutorImpl.h +++ b/dbms/src/Flash/Statistics/CommonExecutorImpl.h @@ -114,4 +114,28 @@ struct TopNImpl static bool isSourceExecutor() { return false; } }; using TopNStatistics = ExecutorStatistics; + +struct CTESinkImpl +{ + static constexpr bool has_extra_info = false; + + static constexpr auto type = "CTESink"; + + static bool isMatch(const tipb::Executor * executor) { return executor->has_cte_sink(); } + + static bool isSourceExecutor() { return false; } +}; +using CTESinkStatistics = ExecutorStatistics; + +struct CTESourceImpl +{ + static constexpr bool has_extra_info = false; + + static constexpr auto type = "CTESource"; + + static bool isMatch(const tipb::Executor * executor) { return executor->has_cte_source(); } + + static bool isSourceExecutor() { return true; } +}; +using CTESourceStatistics = ExecutorStatistics; } // namespace DB diff --git a/dbms/src/Flash/Statistics/ExecutorStatisticsCollector.cpp b/dbms/src/Flash/Statistics/ExecutorStatisticsCollector.cpp index f675680eff8..0a093825880 100644 --- a/dbms/src/Flash/Statistics/ExecutorStatisticsCollector.cpp +++ b/dbms/src/Flash/Statistics/ExecutorStatisticsCollector.cpp @@ -95,7 +95,9 @@ void ExecutorStatisticsCollector::initialize(DAGContext * dag_context_) TableScanStatistics, TopNStatistics, WindowStatistics, - ExpandStatistics>(&executor)) + ExpandStatistics, + CTESinkStatistics, + CTESourceStatistics>(&executor)) { throw TiFlashException( fmt::format("Unknown executor type, executor_id: {}", executor.executor_id()), diff --git a/dbms/src/Flash/Statistics/traverseExecutors.cpp b/dbms/src/Flash/Statistics/traverseExecutors.cpp index c2d217df255..94afde8e405 100644 --- a/dbms/src/Flash/Statistics/traverseExecutors.cpp +++ b/dbms/src/Flash/Statistics/traverseExecutors.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB { @@ -51,6 +52,10 @@ Children getChildren(const tipb::Executor & executor) return Children{&executor.exchange_sender().child()}; case tipb::ExecType::TypeExchangeReceiver: return {}; + case tipb::ExecType::TypeCTESink: + return Children{&executor.cte_sink().child()}; + case tipb::ExecType::TypeCTESource: + return {}; case tipb::ExecType::TypeKill: throw TiFlashException("Kill executor is not supported", Errors::Coprocessor::Unimplemented); default: diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 39eebf16917..f3e50372e53 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -19,11 +19,6 @@ namespace DB { void CTESinkOp::operateSuffixImpl() { - this->cte.reset(); - this->cte_manager->releaseCTE(this->query_id_and_cte_id); - - // In case some tasks are still in WAITING_FOR_NOTIFY status - this->cte->notifyEOF(); LOG_DEBUG(log, "finish write with {} rows", this->total_rows); } diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 632815d6b7e..950ddb15ce9 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -34,7 +34,14 @@ class CTESinkOp : public SinkOp , cte(cte_manager_->getCTE(query_id_and_cte_id_)) {} - ~CTESinkOp() override { assert(!this->cte); } + ~CTESinkOp() override + { + // In case some tasks are still in WAITING_FOR_NOTIFY status + this->cte->notifyEOF(); + + this->cte.reset(); + this->cte_manager->releaseCTE(this->query_id_and_cte_id); + } String getName() const override { return "CTESinkOp"; } bool canHandleSelectiveBlock() const override { return true; } diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index 6f20a238e08..4c43369bab1 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -20,12 +20,9 @@ namespace DB { void CTESourceOp::operateSuffixImpl() { - this->cte.reset(); - this->cte_manager->releaseCTE(this->query_id_and_cte_id); LOG_DEBUG(log, "finish read {} rows from cte source", total_rows); } -// TODO in some cases, source needs to manually filter some data when cte saves all data(start here) OperatorStatus CTESourceOp::readImpl(Block & block) { auto res = this->cte->tryGetBlockAt(this->block_fetch_idx); @@ -38,9 +35,12 @@ OperatorStatus CTESourceOp::readImpl(Block & block) return OperatorStatus::HAS_OUTPUT; case DB::FetchStatus::Waiting: if unlikely (this->block_fetch_idx == 0) + { // CTE has not begun to receive data yet when block_fetch_idx == 0 // So we need to wait the notify from CTE + setNotifyFuture(this->cte.get()); return OperatorStatus::WAIT_FOR_NOTIFY; + } else return OperatorStatus::WAITING; case DB::FetchStatus::Cancelled: diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index e089ceb4312..ee06cf95ec0 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -30,17 +30,26 @@ class CTESourceOp : public SourceOp PipelineExecutorContext & exec_context_, const String & req_id, const String & query_id_and_cte_id_, - CTEManager * cte_manager_) + CTEManager * cte_manager_, + const NamesAndTypes & schema) : SourceOp(exec_context_, req_id) , query_id_and_cte_id(query_id_and_cte_id_) , cte_manager(cte_manager_) , cte(cte_manager_->getCTE(query_id_and_cte_id_)) - {} + { + setHeader(Block(getColumnWithTypeAndName(schema))); + } - ~CTESourceOp() override { assert(!this->cte); } + ~CTESourceOp() override + { + this->cte.reset(); + this->cte_manager->releaseCTE(this->query_id_and_cte_id); + } String getName() const override { return "CTESourceOp"; } + IOProfileInfoPtr getIOProfileInfo() const override { return IOProfileInfo::createForLocal(profile_info_ptr); } + protected: void operateSuffixImpl() override; From 6e903ef4b52f06c81badf478ea9aa223ad348647 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 9 May 2025 18:08:37 +0800 Subject: [PATCH 029/118] fix more bugs --- dbms/src/Flash/Mpp/MPPTask.cpp | 12 +++++++----- dbms/src/Flash/Mpp/MPPTask.h | 2 ++ dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp | 16 ++++++++++++++++ dbms/src/Flash/Planner/PhysicalPlan.cpp | 3 ++- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 2 +- dbms/src/Flash/Statistics/traverseExecutors.cpp | 2 +- dbms/src/Operators/CTESinkOp.h | 4 +++- dbms/src/Operators/Operator.h | 3 +++ 8 files changed, 35 insertions(+), 9 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index c20ccaee0c8..472ed81e692 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -211,6 +211,7 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) throw Exception("Task has either exchange sender or cte sink"); // There is no need to register tunnel for cte sink + this->has_cte_sink = true; return; } auto tunnel_set_local = std::make_shared(log->identifier()); @@ -620,8 +621,9 @@ void MPPTask::runImpl() /// tidb(limit)◄──┼──tiflash(limit)◄─┼─tiflash(no limit) /// └──tiflash(limit)◄─┴─tiflash(no limit) - // finish MPPTunnel - finishWrite(); + if likely (!this->has_cte_sink) + finishWrite(); // finish MPPTunnel + // finish receiver receiver_set->close(); } @@ -816,7 +818,8 @@ int MPPTask::estimateCountOfNewThreads() { auto query_executor = query_executor_holder.tryGet(); RUNTIME_CHECK_MSG( - query_executor && (dag_context->tunnel_set != nullptr || dag_context->dag_request->root_executor().has_cte_sink()), + query_executor + && (dag_context->tunnel_set != nullptr || dag_context->dag_request->root_executor().has_cte_sink()), "It should not estimate the threads for the uninitialized task {}", id.toString()); @@ -825,8 +828,7 @@ int MPPTask::estimateCountOfNewThreads() int external_thread_count = 0; if likely (dag_context->tunnel_set != nullptr) external_thread_count = dag_context->tunnel_set->getExternalThreadCnt(); - return (*query_executor)->estimateNewThreadCount() + 1 + external_thread_count - + new_thread_count_of_mpp_receiver; + return (*query_executor)->estimateNewThreadCount() + 1 + external_thread_count + new_thread_count_of_mpp_receiver; } } // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index 477b7c6e7be..db9330f2c0a 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -199,6 +199,8 @@ class MPPTask MPPReceiverSetPtr receiver_set; + bool has_cte_sink = false; + int new_thread_count_of_mpp_receiver = 0; const LoggerPtr log; diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp index f495edee46f..dba6214adc5 100644 --- a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp @@ -13,9 +13,11 @@ // limitations under the License. #include +#include #include #include #include +#include namespace DB { @@ -132,9 +134,19 @@ OperatorStatus PipelineExec::executeImpl() assert(!io_op); assert(!waiting_for_notify); + bool do_log = false; + if (sink_op->getName() == "CTESinkOp") + do_log = true; + + auto * log = &Poco::Logger::get("LRUCache"); + Block block; + if (do_log) + LOG_INFO(log, fmt::format("xzxdebug executeImpl: beginning {}", block.dumpStructure())); size_t start_transform_op_index = 0; auto op_status = fetchBlock(block, start_transform_op_index); + if (do_log) + LOG_INFO(log, fmt::format("xzxdebug executeImpl: after fetch {}", block.dumpStructure())); // If the status `fetchBlock` returns isn't `HAS_OUTPUT`, it means that `fetchBlock` did not return a block. if (op_status != OperatorStatus::HAS_OUTPUT) return op_status; @@ -148,6 +160,10 @@ OperatorStatus PipelineExec::executeImpl() { const auto & transform_op = transform_ops[transform_op_index]; op_status = transform_op->transform(block); + if (do_log) + LOG_INFO( + log, + fmt::format("xzxdebug executeImpl: after transform {} {}", transform_op_index, block.dumpStructure())); HANDLE_OP_STATUS(transform_op, op_status, OperatorStatus::HAS_OUTPUT); if (block && block.rows() == 0) return OperatorStatus::NEED_INPUT; diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 7ccd95bc548..3ed5eb52fd5 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -303,7 +303,8 @@ PhysicalPlanNodePtr PhysicalPlan::popBack() void PhysicalPlan::addRootFinalProjectionIfNeed() { RUNTIME_CHECK(root_node); - if (root_node->tp() != PlanType::ExchangeSender && root_node->tp() != PlanType::MockExchangeSender) + if (root_node->tp() != PlanType::ExchangeSender && root_node->tp() != PlanType::MockExchangeSender + && root_node->tp() != PlanType::CTESink) { pushBack(root_node); buildFinalProjection(fmt::format("{}_", root_node->execId()), true); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 6ae77656392..e319a88e974 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -33,7 +33,7 @@ PhysicalPlanNodePtr PhysicalCTESource::build( DAGSchema dag_schema; for (int i = 0; i < cte_source.field_types_size(); ++i) { - String name = genNameForExchangeReceiver(i); + String name = genNameForExchangeReceiver(i); // TODO change name TiDB::ColumnInfo info = TiDB::fieldTypeToColumnInfo(cte_source.field_types(i)); dag_schema.emplace_back(std::move(name), std::move(info)); } diff --git a/dbms/src/Flash/Statistics/traverseExecutors.cpp b/dbms/src/Flash/Statistics/traverseExecutors.cpp index 94afde8e405..3af2bb960c8 100644 --- a/dbms/src/Flash/Statistics/traverseExecutors.cpp +++ b/dbms/src/Flash/Statistics/traverseExecutors.cpp @@ -62,4 +62,4 @@ Children getChildren(const tipb::Executor & executor) throw TiFlashException("Should not reach here", Errors::Coprocessor::Internal); } } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 950ddb15ce9..20ce353dfd6 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -38,7 +38,7 @@ class CTESinkOp : public SinkOp { // In case some tasks are still in WAITING_FOR_NOTIFY status this->cte->notifyEOF(); - + this->cte.reset(); this->cte_manager->releaseCTE(this->query_id_and_cte_id); } @@ -46,6 +46,8 @@ class CTESinkOp : public SinkOp String getName() const override { return "CTESinkOp"; } bool canHandleSelectiveBlock() const override { return true; } + void setHeaderImpl() override { std::cout << "1"; } + protected: void operateSuffixImpl() override; OperatorStatus writeImpl(Block && block) override; diff --git a/dbms/src/Operators/Operator.h b/dbms/src/Operators/Operator.h index 5312898848d..399276bc6c5 100644 --- a/dbms/src/Operators/Operator.h +++ b/dbms/src/Operators/Operator.h @@ -84,10 +84,13 @@ class Operator } void setHeader(const Block & header_) { + setHeaderImpl(); assert(header_ && !header); header = header_; } + virtual void setHeaderImpl() {} + const OperatorProfileInfoPtr & getProfileInfo() const { return profile_info_ptr; } virtual IOProfileInfoPtr getIOProfileInfo() const { throw Exception("Unsupport"); } From 7ad85b0382c8033a12159ac3b714a4cba793b793 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 12 May 2025 13:54:44 +0800 Subject: [PATCH 030/118] fix bugs --- .../Flash/Coprocessor/GenSchemaAndColumn.cpp | 5 +++ .../Flash/Coprocessor/GenSchemaAndColumn.h | 1 + dbms/src/Flash/Planner/PhysicalPlan.cpp | 7 ++++ dbms/src/Flash/Planner/PhysicalPlan.h | 2 + .../Flash/Planner/Plans/PhysicalCTESource.cpp | 2 +- .../Planner/Plans/PhysicalProjection.cpp | 42 +++++++++++++++++++ .../Flash/Planner/Plans/PhysicalProjection.h | 5 +++ dbms/src/Operators/CTESinkOp.h | 1 - dbms/src/Operators/Operator.h | 2 - 9 files changed, 63 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp index f1983a242df..43fb498756a 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp @@ -61,6 +61,11 @@ String genNameForExchangeReceiver(Int32 col_index) return fmt::format("exchange_receiver_{}", col_index); } +String genNameForCTESource(Int32 col_index) +{ + return fmt::format("cte_source_{}", col_index); +} + NamesAndTypes genNamesAndTypes(const TiDB::ColumnInfos & column_infos, const StringRef & column_prefix) { NamesAndTypes names_and_types; diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h index 4c579662912..1db05bfd38a 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h @@ -27,6 +27,7 @@ namespace DB NamesAndTypes genNamesAndTypesForExchangeReceiver(const TiDBTableScan & table_scan); NamesAndTypes genNamesAndTypesForTableScan(const TiDBTableScan & table_scan); String genNameForExchangeReceiver(Int32 col_index); +String genNameForCTESource(Int32 col_index); NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan, const StringRef & column_prefix); NamesAndTypes genNamesAndTypes(const TiDB::ColumnInfos & column_infos, const StringRef & column_prefix); diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 3ed5eb52fd5..099feed94e7 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -250,6 +250,7 @@ void PhysicalPlan::build(const tipb::Executor * executor) } case tipb::ExecType::TypeCTESink: { + buildFinalProjectionForCTE(); auto fine_grained_shuffle = FineGrainedShuffle(executor); GET_METRIC(tiflash_coprocessor_executor_count, type_cte_sink).Increment(); pushBack( @@ -278,6 +279,12 @@ void PhysicalPlan::buildFinalProjection(const String & column_prefix, bool is_ro pushBack(final_projection); } +void PhysicalPlan::buildFinalProjectionForCTE() +{ + const auto & final_projection = PhysicalProjection::buildRootFinalForCTE(context, log, popBack()); + pushBack(final_projection); +} + DAGContext & PhysicalPlan::dagContext() const { return *context.getDAGContext(); diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h index e94ca85631d..d7bcdafa4b7 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.h +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -54,6 +54,8 @@ class PhysicalPlan void buildFinalProjection(const String & column_prefix, bool is_root); + void buildFinalProjectionForCTE(); + PhysicalPlanNodePtr popBack(); void pushBack(const PhysicalPlanNodePtr & plan); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index e319a88e974..d8c275083c1 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -33,7 +33,7 @@ PhysicalPlanNodePtr PhysicalCTESource::build( DAGSchema dag_schema; for (int i = 0; i < cte_source.field_types_size(); ++i) { - String name = genNameForExchangeReceiver(i); // TODO change name + String name = genNameForCTESource(i); TiDB::ColumnInfo info = TiDB::fieldTypeToColumnInfo(cte_source.field_types(i)); dag_schema.emplace_back(std::move(name), std::move(info)); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp index 08ca9ef7df8..8ab4dc9043e 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp @@ -23,6 +23,9 @@ #include #include #include +#include + +#include namespace DB { @@ -138,6 +141,45 @@ PhysicalPlanNodePtr PhysicalProjection::buildRootFinal( return physical_projection; } +PhysicalPlanNodePtr PhysicalProjection::buildRootFinalForCTE( + const Context & context, + const LoggerPtr & log, + const PhysicalPlanNodePtr & child) +{ + RUNTIME_CHECK(child); + + const NamesAndTypes & child_schema = child->getSchema(); + DAGExpressionAnalyzer analyzer{child_schema, context}; + ExpressionActionsPtr project_actions = PhysicalPlanHelper::newActions(child->getSampleBlock()); + + NamesWithAliases final_project_aliases; + auto col_num = child_schema.size(); + for (size_t i = 0; i < col_num; i++) + final_project_aliases.push_back(std::make_pair(child_schema[i].name, genNameForCTESource(i))); + + project_actions->add(ExpressionAction::project(final_project_aliases)); + NamesAndTypes schema; + for (size_t i = 0; i < final_project_aliases.size(); ++i) + { + const auto & alias = final_project_aliases[i].second; + RUNTIME_CHECK(!alias.empty()); + const auto & type = analyzer.getCurrentInputColumns()[i].type; + schema.emplace_back(alias, type); + } + + auto physical_projection = std::make_shared( + child->execId(), + schema, + child->getFineGrainedShuffle(), + log->identifier(), + child, + "final projection", + project_actions); + // Final Projection is not a tidb operator, so no need to record profile streams. + physical_projection->notTiDBOperator(); + return physical_projection; +} + void PhysicalProjection::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { child->buildBlockInputStream(pipeline, context, max_streams); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalProjection.h b/dbms/src/Flash/Planner/Plans/PhysicalProjection.h index 3e97eb12899..cdb129e2a22 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalProjection.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalProjection.h @@ -50,6 +50,11 @@ class PhysicalProjection : public PhysicalUnary bool keep_session_timezone_info, const PhysicalPlanNodePtr & child); + static PhysicalPlanNodePtr buildRootFinalForCTE( + const Context & context, + const LoggerPtr & log, + const PhysicalPlanNodePtr & child); + PhysicalProjection( const String & executor_id_, const NamesAndTypes & schema_, diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 20ce353dfd6..4cd244ef80a 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -46,7 +46,6 @@ class CTESinkOp : public SinkOp String getName() const override { return "CTESinkOp"; } bool canHandleSelectiveBlock() const override { return true; } - void setHeaderImpl() override { std::cout << "1"; } protected: void operateSuffixImpl() override; diff --git a/dbms/src/Operators/Operator.h b/dbms/src/Operators/Operator.h index 399276bc6c5..83a9a5958a2 100644 --- a/dbms/src/Operators/Operator.h +++ b/dbms/src/Operators/Operator.h @@ -84,12 +84,10 @@ class Operator } void setHeader(const Block & header_) { - setHeaderImpl(); assert(header_ && !header); header = header_; } - virtual void setHeaderImpl() {} const OperatorProfileInfoPtr & getProfileInfo() const { return profile_info_ptr; } From 4729e8efb9cecc5b06f4d345c23990cca073d574 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 13 May 2025 15:27:34 +0800 Subject: [PATCH 031/118] fix bugs --- dbms/src/Flash/Planner/PhysicalPlan.cpp | 11 ++++++--- dbms/src/Flash/Planner/PhysicalPlan.h | 2 +- .../Planner/Plans/PhysicalProjection.cpp | 23 ++++++++++++++++--- .../Flash/Planner/Plans/PhysicalProjection.h | 4 +++- 4 files changed, 32 insertions(+), 8 deletions(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 099feed94e7..877e4725e70 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -250,7 +250,7 @@ void PhysicalPlan::build(const tipb::Executor * executor) } case tipb::ExecType::TypeCTESink: { - buildFinalProjectionForCTE(); + buildFinalProjectionForCTE(executor->cte_sink()); auto fine_grained_shuffle = FineGrainedShuffle(executor); GET_METRIC(tiflash_coprocessor_executor_count, type_cte_sink).Increment(); pushBack( @@ -279,9 +279,14 @@ void PhysicalPlan::buildFinalProjection(const String & column_prefix, bool is_ro pushBack(final_projection); } -void PhysicalPlan::buildFinalProjectionForCTE() +void PhysicalPlan::buildFinalProjectionForCTE(const tipb::CTESink & sink) { - const auto & final_projection = PhysicalProjection::buildRootFinalForCTE(context, log, popBack()); + const auto & final_projection = PhysicalProjection::buildRootFinalForCTE( + context, + log, + popBack(), + sink, + dagContext().keep_session_timezone_info); pushBack(final_projection); } diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h index d7bcdafa4b7..1668595eab2 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.h +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -54,7 +54,7 @@ class PhysicalPlan void buildFinalProjection(const String & column_prefix, bool is_root); - void buildFinalProjectionForCTE(); + void buildFinalProjectionForCTE(const tipb::CTESink &); PhysicalPlanNodePtr popBack(); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp index 8ab4dc9043e..28b9a1dfba3 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp @@ -17,13 +17,13 @@ #include #include #include +#include #include #include #include #include #include #include -#include #include @@ -144,7 +144,9 @@ PhysicalPlanNodePtr PhysicalProjection::buildRootFinal( PhysicalPlanNodePtr PhysicalProjection::buildRootFinalForCTE( const Context & context, const LoggerPtr & log, - const PhysicalPlanNodePtr & child) + const PhysicalPlanNodePtr & child, + const tipb::CTESink & sink, + bool keep_session_timezone_info) { RUNTIME_CHECK(child); @@ -152,10 +154,25 @@ PhysicalPlanNodePtr PhysicalProjection::buildRootFinalForCTE( DAGExpressionAnalyzer analyzer{child_schema, context}; ExpressionActionsPtr project_actions = PhysicalPlanHelper::newActions(child->getSampleBlock()); + auto required_schema_size = sink.field_types_size(); + std::vector required_schema; + required_schema.reserve(required_schema_size); + std::vector output_offsets; + output_offsets.reserve(required_schema_size); + for (int i = 0; i < required_schema_size; i++) + { + required_schema.push_back(sink.field_types(i)); + output_offsets.push_back(i); + } + + NamesWithAliases project_aliases + = analyzer + .buildFinalProjection(project_actions, required_schema, output_offsets, "", keep_session_timezone_info); NamesWithAliases final_project_aliases; auto col_num = child_schema.size(); + RUNTIME_CHECK(required_schema_size == static_cast(col_num)); for (size_t i = 0; i < col_num; i++) - final_project_aliases.push_back(std::make_pair(child_schema[i].name, genNameForCTESource(i))); + final_project_aliases.push_back(std::make_pair(project_aliases[i].first, genNameForCTESource(i))); project_actions->add(ExpressionAction::project(final_project_aliases)); NamesAndTypes schema; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalProjection.h b/dbms/src/Flash/Planner/Plans/PhysicalProjection.h index cdb129e2a22..7b651266a2f 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalProjection.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalProjection.h @@ -53,7 +53,9 @@ class PhysicalProjection : public PhysicalUnary static PhysicalPlanNodePtr buildRootFinalForCTE( const Context & context, const LoggerPtr & log, - const PhysicalPlanNodePtr & child); + const PhysicalPlanNodePtr & child, + const tipb::CTESink & sink, + bool keep_session_timezone_info); PhysicalProjection( const String & executor_id_, From 1052c613eca32a1ad5d5189961138b7a90e6d4f9 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 13 May 2025 17:13:52 +0800 Subject: [PATCH 032/118] moderate refactor --- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 4 +- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 33 +++++++--- dbms/src/Operators/CTE.cpp | 20 ++---- dbms/src/Operators/CTE.h | 4 +- dbms/src/Operators/CTEReader.cpp | 56 ++++++++++++++++ dbms/src/Operators/CTEReader.h | 66 +++++++++++++++++++ dbms/src/Operators/CTESource.cpp | 26 ++++---- dbms/src/Operators/CTESource.h | 23 ++----- 8 files changed, 174 insertions(+), 58 deletions(-) create mode 100644 dbms/src/Operators/CTEReader.cpp create mode 100644 dbms/src/Operators/CTEReader.h diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 15e8bf0265d..a70b4f3dcde 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -51,7 +51,9 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( size_t partition_id = 0; String query_id_and_cte_id_prefix = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); group_builder.transform([&](auto & builder) { - String query_id_and_cte_id = fmt::format("{}_{}", query_id_and_cte_id_prefix, partition_id); + String query_id_and_cte_id = query_id_and_cte_id_prefix; + if (fine_grained_shuffle.enabled()) + query_id_and_cte_id = fmt::format("{}_{}", query_id_and_cte_id_prefix, partition_id); builder.setSinkOp( std::make_unique(exec_context, log->identifier(), query_id_and_cte_id, context.getCTEManager())); partition_id++; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index d8c275083c1..17965b56fce 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -18,8 +18,10 @@ #include #include #include +#include #include +#include namespace DB { @@ -57,18 +59,31 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( if (fine_grained_shuffle.enabled()) concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); - String query_id_and_cte_id_prefix = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); + String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); - for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) + if (fine_grained_shuffle.enabled()) + { + for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) + { + group_builder.addConcurrency(std::make_unique( + exec_context, + log->identifier(), + std::make_shared( + fmt::format("{}_{}", query_id_and_cte_id, partition_id), + context.getCTEManager()), + schema)); + } + } + else { - String query_id_and_cte_id = fmt::format("{}_{}", query_id_and_cte_id_prefix, partition_id); - group_builder.addConcurrency(std::make_unique( - exec_context, - log->identifier(), - query_id_and_cte_id, - context.getCTEManager(), - schema)); + auto cte_reader = std::make_shared(query_id_and_cte_id, context.getCTEManager()); + for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) + { + group_builder.addConcurrency( + std::make_unique(exec_context, log->identifier(), cte_reader, schema)); + } } + context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 3569013e8a7..6c898be272e 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -14,13 +14,14 @@ #include +#include #include #include #include namespace DB { -FetchStatus CTE::checkAvailableBlockAt(size_t idx) +FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) { std::shared_lock lock(this->rw_lock); auto block_num = this->blocks.size(); @@ -31,21 +32,10 @@ FetchStatus CTE::checkAvailableBlockAt(size_t idx) else return FetchStatus::Waiting; } - return FetchStatus::Ok; -} -std::pair CTE::tryGetBlockAt(size_t idx) -{ - std::shared_lock lock(this->rw_lock); - auto block_num = this->blocks.size(); - if (block_num <= idx) - { - if (this->is_eof) - return {FetchStatus::Eof, Block()}; - else - return {FetchStatus::Waiting, Block()}; - } - return {FetchStatus::Ok, this->blocks[idx]}; + for (size_t i = 0; i < block_num; i++) + queue.push_back(this->blocks[i]); + return FetchStatus::Ok; } void CTE::pushBlock(const Block & block) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 18d05626958..3b81042aa3f 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -20,7 +20,6 @@ #include #include -#include namespace DB { @@ -37,8 +36,7 @@ class CTE : public NotifyFuture public: ~CTE() override = default; - std::pair tryGetBlockAt(size_t idx); - FetchStatus checkAvailableBlockAt(size_t idx); + FetchStatus tryGetBunchBlocks(size_t idx, std::deque & queue); void pushBlock(const Block & block); void notifyEOF(); diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp new file mode 100644 index 00000000000..a71ac513266 --- /dev/null +++ b/dbms/src/Operators/CTEReader.cpp @@ -0,0 +1,56 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +#include + +namespace DB +{ +std::pair CTEReader::fetchNextBlock() +{ + std::lock_guard lock(this->mu); + if (!this->blocks.empty()) + { + auto & block = this->blocks.front(); + this->blocks.pop_front(); + return {FetchStatus::Ok, block}; + } + + auto ret = this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); + switch (ret) + { + case FetchStatus::Waiting: + case FetchStatus::Eof: + case FetchStatus::Cancelled: + return {ret, Block()}; + case FetchStatus::Ok: + this->block_fetch_idx += this->blocks.size(); + auto & block = this->blocks.front(); + this->blocks.pop_front(); + return {ret, block}; + } + throw Exception("Should not reach here"); +} + +FetchStatus CTEReader::checkAvailableBlock() +{ + std::lock_guard lock(this->mu); + if (!this->blocks.empty()) + return FetchStatus::Ok; + + return this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); +} +} // namespace DB diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h new file mode 100644 index 00000000000..6787c7abe5f --- /dev/null +++ b/dbms/src/Operators/CTEReader.h @@ -0,0 +1,66 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include + +#include +#include + +namespace DB +{ +class CTEReader +{ +public: + CTEReader( + const String & query_id_and_cte_id_, + CTEManager * cte_manager_) + : query_id_and_cte_id(query_id_and_cte_id_) + , cte_manager(cte_manager_) + , cte(cte_manager_->getCTE(query_id_and_cte_id_)) + {} + + ~CTEReader() + { + this->cte.reset(); + this->cte_manager->releaseCTE(this->query_id_and_cte_id); + } + + std::pair fetchNextBlock(); + FetchStatus checkAvailableBlock(); + + bool isBlockGenerated() + { + std::lock_guard lock(this->mu); + + // `block_fetch_idx == 0` means that CTE hasn't received block yet, maybe it is waiting + // for the finish of join executor + return this->block_fetch_idx != 0; + } + + void setNotifyFuture() { ::DB::setNotifyFuture(cte.get()); } + +private: + String query_id_and_cte_id; + CTEManager * cte_manager; + std::shared_ptr cte; + + std::mutex mu; + std::deque blocks; + size_t block_fetch_idx = 0; +}; +} // namespace DB diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index 4c43369bab1..54cdb74fcc7 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -25,37 +25,37 @@ void CTESourceOp::operateSuffixImpl() OperatorStatus CTESourceOp::readImpl(Block & block) { - auto res = this->cte->tryGetBlockAt(this->block_fetch_idx); + auto res = this->cte_reader->fetchNextBlock(); switch (res.first) { - case DB::FetchStatus::Eof: - case DB::FetchStatus::Ok: + case FetchStatus::Eof: + case FetchStatus::Ok: block = res.second; - ++(this->block_fetch_idx); return OperatorStatus::HAS_OUTPUT; - case DB::FetchStatus::Waiting: - if unlikely (this->block_fetch_idx == 0) + case FetchStatus::Waiting: + if likely (this->cte_reader->isBlockGenerated()) + { + return OperatorStatus::WAITING; + } + else { - // CTE has not begun to receive data yet when block_fetch_idx == 0 + // CTE has not begun to receive data yet // So we need to wait the notify from CTE - setNotifyFuture(this->cte.get()); + this->cte_reader->setNotifyFuture(); return OperatorStatus::WAIT_FOR_NOTIFY; } - else - return OperatorStatus::WAITING; - case DB::FetchStatus::Cancelled: + case FetchStatus::Cancelled: return OperatorStatus::CANCELLED; } } OperatorStatus CTESourceOp::awaitImpl() { - auto res = this->cte->checkAvailableBlockAt(this->block_fetch_idx); + auto res = this->cte_reader->checkAvailableBlock(); switch (res) { case DB::FetchStatus::Eof: case DB::FetchStatus::Ok: - // Do not add block_fetch_idx here, as we just judge if there are available blocks return OperatorStatus::HAS_OUTPUT; case DB::FetchStatus::Waiting: return OperatorStatus::WAITING; diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index ee06cf95ec0..7c3c8de34f4 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -20,6 +20,9 @@ #include #include #include +#include + +#include namespace DB { @@ -29,23 +32,14 @@ class CTESourceOp : public SourceOp CTESourceOp( PipelineExecutorContext & exec_context_, const String & req_id, - const String & query_id_and_cte_id_, - CTEManager * cte_manager_, + std::shared_ptr cte_reader_, const NamesAndTypes & schema) : SourceOp(exec_context_, req_id) - , query_id_and_cte_id(query_id_and_cte_id_) - , cte_manager(cte_manager_) - , cte(cte_manager_->getCTE(query_id_and_cte_id_)) + , cte_reader(cte_reader_) { setHeader(Block(getColumnWithTypeAndName(schema))); } - ~CTESourceOp() override - { - this->cte.reset(); - this->cte_manager->releaseCTE(this->query_id_and_cte_id); - } - String getName() const override { return "CTESourceOp"; } IOProfileInfoPtr getIOProfileInfo() const override { return IOProfileInfo::createForLocal(profile_info_ptr); } @@ -58,12 +52,7 @@ class CTESourceOp : public SourceOp OperatorStatus awaitImpl() override; private: - String query_id_and_cte_id; - CTEManager * cte_manager; - std::shared_ptr cte; - size_t block_fetch_idx = 0; - + std::shared_ptr cte_reader; uint64_t total_rows{}; - std::queue block_queue; }; } // namespace DB From e075e684097b84bb872c6a5e5101a7d40b1664a8 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 13 May 2025 19:25:43 +0800 Subject: [PATCH 033/118] fix bug --- contrib/tipb | 2 +- dbms/src/Operators/CTEReader.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/tipb b/contrib/tipb index d2c561dad34..83bf5e992de 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit d2c561dad347b35ef1d7ee2f57ecd9f3123bbdcf +Subproject commit 83bf5e992de64052225f5e718058bc2769879123 diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index a71ac513266..9166f7ae3f0 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -24,7 +24,7 @@ std::pair CTEReader::fetchNextBlock() std::lock_guard lock(this->mu); if (!this->blocks.empty()) { - auto & block = this->blocks.front(); + auto block = this->blocks.front(); this->blocks.pop_front(); return {FetchStatus::Ok, block}; } @@ -38,7 +38,7 @@ std::pair CTEReader::fetchNextBlock() return {ret, Block()}; case FetchStatus::Ok: this->block_fetch_idx += this->blocks.size(); - auto & block = this->blocks.front(); + auto block = this->blocks.front(); this->blocks.pop_front(); return {ret, block}; } From ccca5ab8b79487b654ed22310d49237236324eb6 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 13 May 2025 19:26:49 +0800 Subject: [PATCH 034/118] Revert "fix bug" This reverts commit e075e684097b84bb872c6a5e5101a7d40b1664a8. --- contrib/tipb | 2 +- dbms/src/Operators/CTEReader.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/tipb b/contrib/tipb index 83bf5e992de..d2c561dad34 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit 83bf5e992de64052225f5e718058bc2769879123 +Subproject commit d2c561dad347b35ef1d7ee2f57ecd9f3123bbdcf diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 9166f7ae3f0..a71ac513266 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -24,7 +24,7 @@ std::pair CTEReader::fetchNextBlock() std::lock_guard lock(this->mu); if (!this->blocks.empty()) { - auto block = this->blocks.front(); + auto & block = this->blocks.front(); this->blocks.pop_front(); return {FetchStatus::Ok, block}; } @@ -38,7 +38,7 @@ std::pair CTEReader::fetchNextBlock() return {ret, Block()}; case FetchStatus::Ok: this->block_fetch_idx += this->blocks.size(); - auto block = this->blocks.front(); + auto & block = this->blocks.front(); this->blocks.pop_front(); return {ret, block}; } From 239381707a6a27f654d7062a5bf33508dfddb428 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 13 May 2025 19:27:27 +0800 Subject: [PATCH 035/118] fix bugs --- dbms/src/Operators/CTEReader.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index a71ac513266..9166f7ae3f0 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -24,7 +24,7 @@ std::pair CTEReader::fetchNextBlock() std::lock_guard lock(this->mu); if (!this->blocks.empty()) { - auto & block = this->blocks.front(); + auto block = this->blocks.front(); this->blocks.pop_front(); return {FetchStatus::Ok, block}; } @@ -38,7 +38,7 @@ std::pair CTEReader::fetchNextBlock() return {ret, Block()}; case FetchStatus::Ok: this->block_fetch_idx += this->blocks.size(); - auto & block = this->blocks.front(); + auto block = this->blocks.front(); this->blocks.pop_front(); return {ret, block}; } From 5f0ace7e41c3551cbabbc7fee271d32a1b499291 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 13 May 2025 19:37:34 +0800 Subject: [PATCH 036/118] fix bug --- dbms/src/Operators/CTE.cpp | 5 ++++- dbms/src/Operators/CTEReader.cpp | 15 +++++++++++++-- dbms/src/Operators/CTEReader.h | 6 ++---- dbms/src/Operators/CTESource.h | 2 +- 4 files changed, 20 insertions(+), 8 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 6c898be272e..a0d6e183e0c 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -14,6 +14,7 @@ #include +#include #include #include #include @@ -23,6 +24,8 @@ namespace DB { FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) { + assert(queue.empty()); + std::shared_lock lock(this->rw_lock); auto block_num = this->blocks.size(); if (block_num <= idx) @@ -33,7 +36,7 @@ FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) return FetchStatus::Waiting; } - for (size_t i = 0; i < block_num; i++) + for (size_t i = idx; i < block_num; i++) queue.push_back(this->blocks[i]); return FetchStatus::Ok; } diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 9166f7ae3f0..295fc9821d0 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include +#include #include @@ -51,6 +51,17 @@ FetchStatus CTEReader::checkAvailableBlock() if (!this->blocks.empty()) return FetchStatus::Ok; - return this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); + auto ret = this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); + switch (ret) + { + case FetchStatus::Waiting: + case FetchStatus::Cancelled: + case FetchStatus::Eof: + return ret; + case FetchStatus::Ok: + this->block_fetch_idx += this->blocks.size(); + return FetchStatus::Ok; + } + throw Exception("Should not reach here"); } } // namespace DB diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 6787c7abe5f..292ee0f77d6 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -14,9 +14,9 @@ #pragma once -#include #include #include +#include #include #include @@ -26,9 +26,7 @@ namespace DB class CTEReader { public: - CTEReader( - const String & query_id_and_cte_id_, - CTEManager * cte_manager_) + CTEReader(const String & query_id_and_cte_id_, CTEManager * cte_manager_) : query_id_and_cte_id(query_id_and_cte_id_) , cte_manager(cte_manager_) , cte(cte_manager_->getCTE(query_id_and_cte_id_)) diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 7c3c8de34f4..7d9f782b871 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -19,8 +19,8 @@ #include #include #include -#include #include +#include #include From 2a2d6677a178df0614f150a4945a9086afebd7e2 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 15 May 2025 16:49:19 +0800 Subject: [PATCH 037/118] save --- dbms/src/Flash/Coprocessor/DAGContext.h | 11 ++++ .../Flash/Executor/PipelineExecutorContext.h | 6 ++ dbms/src/Flash/Mpp/CTEManager.cpp | 56 ++++++++++++++++--- dbms/src/Flash/Mpp/CTEManager.h | 23 ++++++-- dbms/src/Flash/Mpp/MPPTask.cpp | 28 +++++++--- dbms/src/Flash/Mpp/MPPTaskStatistics.h | 1 - .../Flash/Planner/Plans/PhysicalCTESink.cpp | 15 +++-- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 7 +-- dbms/src/Operators/CTE.cpp | 7 ++- dbms/src/Operators/CTE.h | 19 ++++++- dbms/src/Operators/CTEReader.cpp | 8 ++- dbms/src/Operators/CTEReader.h | 20 ++++++- dbms/src/Operators/CTESinkOp.cpp | 1 - dbms/src/Operators/CTESinkOp.h | 9 ++- dbms/src/Operators/CTESource.cpp | 3 + dbms/src/Operators/CTESource.h | 5 +- dbms/src/Operators/IOProfileInfo.h | 2 +- 17 files changed, 175 insertions(+), 46 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 5179fda8c55..fbb1650ccfc 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -23,6 +23,7 @@ #pragma GCC diagnostic pop #include +#include #include #include #include @@ -359,6 +360,14 @@ class DAGContext MPPReceiverSetPtr getMPPReceiverSet() const { return mpp_receiver_set; } + String getQueryIDAndCTEID() const noexcept { return this->query_id_and_cte_id; } + void setQueryIDAndCTEID(const String & query_id_and_cte_id) + { + // MPP Task has only one CTESink, it's impossible to set query_id_and_cte_id twice + RUNTIME_CHECK(this->query_id_and_cte_id.empty()); + this->query_id_and_cte_id = query_id_and_cte_id; + } + public: DAGRequest dag_request; /// Some existing code inherited from Clickhouse assume that each query must have a valid query string and query ast, @@ -475,6 +484,8 @@ class DAGContext UInt64 connection_id; // It's the session alias between mysql client and tidb String connection_alias; + + String query_id_and_cte_id; }; } // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.h b/dbms/src/Flash/Executor/PipelineExecutorContext.h index 7b1c8f42c89..736ba937c77 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.h +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -143,6 +144,11 @@ class PipelineExecutorContext : private boost::noncopyable void addOneTimeFuture(const OneTimeNotifyFuturePtr & future); + void setQueryIDAndCTEID(const String & query_id_and_cte_id) + { + this->dag_context->setQueryIDAndCTEID(query_id_and_cte_id); + } + private: bool setExceptionPtr(const std::exception_ptr & exception_ptr_); diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index f2d054c0ab9..60635c5d75b 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -14,34 +14,76 @@ #include #include +#include + +#include +#include namespace DB { -std::shared_ptr CTEManager::getCTE(const String & query_id_and_cte_id) +std::shared_ptr CTEManager::getCTE(const String & query_id_and_cte_id, const String & partition_id) { std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if (iter == this->ctes.end()) + this->ctes[query_id_and_cte_id] = std::map{}; + + auto iter_for_cte = this->ctes[query_id_and_cte_id].find(partition_id); + if (iter_for_cte == this->ctes[query_id_and_cte_id].end()) { // It's the first time we request for the specific cte // Create it because no one created it before. auto cte = std::make_shared(); - this->ctes[query_id_and_cte_id] = std::make_pair(1, cte); + this->ctes[query_id_and_cte_id].insert(std::make_pair(partition_id, CTEWithCounter(cte, 1))); return cte; } - ++(iter->second.first); - return iter->second.second; + ++(iter_for_cte->second.counter); + return iter_for_cte->second.cte; } -void CTEManager::releaseCTE(const String & query_id_and_cte_id) +void CTEManager::releaseCTE(const String & query_id_and_cte_id, const String & partition_id) { std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if unlikely (iter == this->ctes.end()) throw Exception(fmt::format("Can't find cte: {}", query_id_and_cte_id)); - --(iter->second.first); - if (iter->second.first == 0) + auto iter_for_cte = iter->second.find(partition_id); + if unlikely (iter_for_cte == iter->second.end()) + throw Exception(fmt::format("Can't find cte: {}, partition: {}", query_id_and_cte_id, partition_id)); + --(iter_for_cte->second.counter); + if (iter_for_cte->second.counter == 0) + iter->second.erase(iter_for_cte); + + if (iter->second.size() == 0) this->ctes.erase(iter); } + +void CTEManager::setRespAndNotifyEOF(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) +{ + this->executeOnManyCTEs(resp, query_id_and_cte_id); +} + +void CTEManager::notifyEOF(const String & query_id_and_cte_id) +{ + this->executeOnManyCTEs(tipb::SelectResponse(), query_id_and_cte_id); +} + +template +void CTEManager::executeOnManyCTEs(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) +{ + std::lock_guard lock(this->mu); + auto iter = this->ctes.find(query_id_and_cte_id); + if unlikely (iter == this->ctes.end()) + throw Exception(fmt::format("Can't find cte: {}", query_id_and_cte_id)); + auto iter_for_cte = iter->second.begin(); + while (iter_for_cte != iter->second.end()) + { + if constexpr (set_resp) + iter_for_cte->second.cte->setRespAndNotifyEOF(resp); + else + iter_for_cte->second.cte->notifyEOF(); + iter_for_cte++; + } +} } // namespace DB diff --git a/dbms/src/Flash/Mpp/CTEManager.h b/dbms/src/Flash/Mpp/CTEManager.h index d062bee0b53..1a6a0d5ba08 100644 --- a/dbms/src/Flash/Mpp/CTEManager.h +++ b/dbms/src/Flash/Mpp/CTEManager.h @@ -18,18 +18,33 @@ #include #include -#include namespace DB { +struct CTEWithCounter +{ + CTEWithCounter(std::shared_ptr cte_, Int32 counter_) + : cte(cte_) + , counter(counter_) + {} + std::shared_ptr cte; + Int32 counter; +}; + +// TODO Test this class with UT class CTEManager { public: - std::shared_ptr getCTE(const String & query_id_and_cte_id); - void releaseCTE(const String & query_id_and_cte_id); + std::shared_ptr getCTE(const String & query_id_and_cte_id, const String & partition_id); + void releaseCTE(const String & query_id_and_cte_id, const String & partition_id); + void setRespAndNotifyEOF(const tipb::SelectResponse & resp, const String & query_id_and_cte_id); + void notifyEOF(const String & query_id_and_cte_id); private: + template + void executeOnManyCTEs(const tipb::SelectResponse & resp, const String & query_id_and_cte_id); + std::mutex mu; - std::map>> ctes; + std::map> ctes; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 472ed81e692..647de9c0a16 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -191,11 +191,25 @@ void MPPTask::abortQueryExecutor() void MPPTask::finishWrite() { - RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); - if (dag_context->collect_execution_summaries - && !ReportExecutionSummaryToCoordinator(meta.mpp_version(), meta.report_execution_summary())) - tunnel_set->sendExecutionSummary(mpp_task_statistics.genExecutionSummaryResponse()); - tunnel_set->finishWrite(); + if (this->has_cte_sink) + { + const String & query_id_and_cte_id = this->dag_context->getQueryIDAndCTEID(); + if (dag_context->collect_execution_summaries) + { + const tipb::SelectResponse & resp = mpp_task_statistics.genExecutionSummaryResponse(); + this->context->getCTEManager()->setRespAndNotifyEOF(resp, query_id_and_cte_id); + } + else + this->context->getCTEManager()->notifyEOF(query_id_and_cte_id); + } + else + { + RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); + if (dag_context->collect_execution_summaries + && !ReportExecutionSummaryToCoordinator(meta.mpp_version(), meta.report_execution_summary())) + tunnel_set->sendExecutionSummary(mpp_task_statistics.genExecutionSummaryResponse()); + tunnel_set->finishWrite(); + } } void MPPTask::run() @@ -621,8 +635,8 @@ void MPPTask::runImpl() /// tidb(limit)◄──┼──tiflash(limit)◄─┼─tiflash(no limit) /// └──tiflash(limit)◄─┴─tiflash(no limit) - if likely (!this->has_cte_sink) - finishWrite(); // finish MPPTunnel + // finish MPPTunnel + finishWrite(); // finish receiver receiver_set->close(); diff --git a/dbms/src/Flash/Mpp/MPPTaskStatistics.h b/dbms/src/Flash/Mpp/MPPTaskStatistics.h index ef012210407..50db59becbd 100644 --- a/dbms/src/Flash/Mpp/MPPTaskStatistics.h +++ b/dbms/src/Flash/Mpp/MPPTaskStatistics.h @@ -25,7 +25,6 @@ #include #include -#include namespace DB { diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index a70b4f3dcde..1ccd8848477 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -19,6 +19,7 @@ #include #include +#include namespace DB { @@ -49,13 +50,15 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( size_t /*concurrency*/) { size_t partition_id = 0; - String query_id_and_cte_id_prefix = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); + String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); + group_builder.transform([&](auto & builder) { - String query_id_and_cte_id = query_id_and_cte_id_prefix; - if (fine_grained_shuffle.enabled()) - query_id_and_cte_id = fmt::format("{}_{}", query_id_and_cte_id_prefix, partition_id); - builder.setSinkOp( - std::make_unique(exec_context, log->identifier(), query_id_and_cte_id, context.getCTEManager())); + builder.setSinkOp(std::make_unique( + exec_context, + log->identifier(), + query_id_and_cte_id, + fine_grained_shuffle.enabled() ? std::to_string(partition_id) : "", + context.getCTEManager())); partition_id++; }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 17965b56fce..4cd487d6848 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -22,6 +22,7 @@ #include #include +#include namespace DB { @@ -68,15 +69,13 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( group_builder.addConcurrency(std::make_unique( exec_context, log->identifier(), - std::make_shared( - fmt::format("{}_{}", query_id_and_cte_id, partition_id), - context.getCTEManager()), + std::make_shared(query_id_and_cte_id, std::to_string(partition_id), context.getCTEManager()), schema)); } } else { - auto cte_reader = std::make_shared(query_id_and_cte_id, context.getCTEManager()); + auto cte_reader = std::make_shared(query_id_and_cte_id, "", context.getCTEManager()); for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) { group_builder.addConcurrency( diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index a0d6e183e0c..b099984e61f 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -51,9 +51,12 @@ void CTE::pushBlock(const Block & block) this->blocks.push_back(block); } -void CTE::notifyEOF() +template +void CTE::notifyEOFImpl() { - std::unique_lock lock(this->rw_lock); + std::unique_lock lock(this->rw_lock, std::defer_lock); + if constexpr (has_lock) + lock.lock(); this->is_eof = true; // Just in case someone is in WAITING_FOR_NOTIFY status diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 3b81042aa3f..6b42e2c83a9 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -18,6 +18,7 @@ #include #include #include +#include #include @@ -38,10 +39,23 @@ class CTE : public NotifyFuture FetchStatus tryGetBunchBlocks(size_t idx, std::deque & queue); void pushBlock(const Block & block); - void notifyEOF(); + void notifyEOF() { this->notifyEOFImpl(); } + void notifyEOFNoLock() { this->notifyEOFImpl(); } + + template + void notifyEOFImpl(); void registerTask(TaskPtr && task) override; + void setRespAndNotifyEOF(const tipb::SelectResponse & resp) + { + std::unique_lock lock(this->rw_lock); + this->resp = resp; + this->notifyEOFNoLock(); + } + + tipb::SelectResponse getResp() const noexcept { return this->resp; } + private: // Return true if CTE has data inline bool hasDataNoLock() const { return !this->blocks.empty(); } @@ -55,6 +69,9 @@ class CTE : public NotifyFuture std::deque waiting_tasks; PipeConditionVariable pipe_cv; + // TODO eof can be set only when execution summary has been sent bool is_eof = false; + + tipb::SelectResponse resp; }; } // namespace DB diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 295fc9821d0..e7680bcade5 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -24,7 +24,7 @@ std::pair CTEReader::fetchNextBlock() std::lock_guard lock(this->mu); if (!this->blocks.empty()) { - auto block = this->blocks.front(); + Block block = std::move(this->blocks.front()); this->blocks.pop_front(); return {FetchStatus::Ok, block}; } @@ -32,13 +32,15 @@ std::pair CTEReader::fetchNextBlock() auto ret = this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); switch (ret) { - case FetchStatus::Waiting: case FetchStatus::Eof: + if (this->resp.execution_summaries_size() == 0) + this->resp = this->cte->getResp(); + case FetchStatus::Waiting: case FetchStatus::Cancelled: return {ret, Block()}; case FetchStatus::Ok: this->block_fetch_idx += this->blocks.size(); - auto block = this->blocks.front(); + Block block = std::move(this->blocks.front()); this->blocks.pop_front(); return {ret, block}; } diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 292ee0f77d6..59c03d94fba 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -26,21 +27,30 @@ namespace DB class CTEReader { public: - CTEReader(const String & query_id_and_cte_id_, CTEManager * cte_manager_) + CTEReader(const String & query_id_and_cte_id_, const String & partition_id_, CTEManager * cte_manager_) : query_id_and_cte_id(query_id_and_cte_id_) + , partition_id(partition_id_) , cte_manager(cte_manager_) - , cte(cte_manager_->getCTE(query_id_and_cte_id_)) + , cte(cte_manager_->getCTE(query_id_and_cte_id_, partition_id)) {} ~CTEReader() { this->cte.reset(); - this->cte_manager->releaseCTE(this->query_id_and_cte_id); + this->cte_manager->releaseCTE(this->query_id_and_cte_id, this->partition_id); } std::pair fetchNextBlock(); FetchStatus checkAvailableBlock(); + void getResp(tipb::SelectResponse & resp) + { + std::lock_guard lock(this->mu); + if (this->resp_fetched) + return; + resp.CopyFrom(this->resp); + } + bool isBlockGenerated() { std::lock_guard lock(this->mu); @@ -54,11 +64,15 @@ class CTEReader private: String query_id_and_cte_id; + String partition_id; CTEManager * cte_manager; std::shared_ptr cte; std::mutex mu; std::deque blocks; size_t block_fetch_idx = 0; + + bool resp_fetched = false; + tipb::SelectResponse resp; }; } // namespace DB diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index f3e50372e53..331e9b1ce41 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -27,7 +27,6 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) if (!block) { this->input_done = true; - this->cte->notifyEOF(); return OperatorStatus::FINISHED; } this->total_rows += block.rows(); diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 4cd244ef80a..0766d93f505 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -27,20 +27,18 @@ class CTESinkOp : public SinkOp PipelineExecutorContext & exec_context_, const String & req_id, const String & query_id_and_cte_id_, + const String & partition_id_, CTEManager * cte_manager_) : SinkOp(exec_context_, req_id) , query_id_and_cte_id(query_id_and_cte_id_) , cte_manager(cte_manager_) - , cte(cte_manager_->getCTE(query_id_and_cte_id_)) + , cte(cte_manager_->getCTE(query_id_and_cte_id_, partition_id_)) {} ~CTESinkOp() override { - // In case some tasks are still in WAITING_FOR_NOTIFY status - this->cte->notifyEOF(); - this->cte.reset(); - this->cte_manager->releaseCTE(this->query_id_and_cte_id); + this->cte_manager->releaseCTE(this->query_id_and_cte_id, this->partition_id); } String getName() const override { return "CTESinkOp"; } @@ -53,6 +51,7 @@ class CTESinkOp : public SinkOp private: String query_id_and_cte_id; + String partition_id; CTEManager * cte_manager; std::shared_ptr cte; size_t total_rows = 0; diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index 54cdb74fcc7..671c52d5fc2 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -29,6 +29,9 @@ OperatorStatus CTESourceOp::readImpl(Block & block) switch (res.first) { case FetchStatus::Eof: + this->cte_reader->getResp(this->resp); + if (this->resp.execution_summaries_size() != 0) + this->io_profile_info->remote_execution_summary.add(this->resp); case FetchStatus::Ok: block = res.second; return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 7d9f782b871..1762e506024 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -36,13 +36,14 @@ class CTESourceOp : public SourceOp const NamesAndTypes & schema) : SourceOp(exec_context_, req_id) , cte_reader(cte_reader_) + , io_profile_info(IOProfileInfo::createForRemote(profile_info_ptr, 1)) { setHeader(Block(getColumnWithTypeAndName(schema))); } String getName() const override { return "CTESourceOp"; } - IOProfileInfoPtr getIOProfileInfo() const override { return IOProfileInfo::createForLocal(profile_info_ptr); } + IOProfileInfoPtr getIOProfileInfo() const override { return io_profile_info; } protected: void operateSuffixImpl() override; @@ -54,5 +55,7 @@ class CTESourceOp : public SourceOp private: std::shared_ptr cte_reader; uint64_t total_rows{}; + IOProfileInfoPtr io_profile_info; + tipb::SelectResponse resp; }; } // namespace DB diff --git a/dbms/src/Operators/IOProfileInfo.h b/dbms/src/Operators/IOProfileInfo.h index 2515096ad44..429b470dec3 100644 --- a/dbms/src/Operators/IOProfileInfo.h +++ b/dbms/src/Operators/IOProfileInfo.h @@ -63,7 +63,7 @@ struct IOProfileInfo OperatorProfileInfoPtr operator_info; const bool is_local; - std::vector connection_profile_infos{}; + std::vector connection_profile_infos; RemoteExecutionSummary remote_execution_summary{}; }; } // namespace DB From 9ff7f33c90332e4e971cf4a20992bce08f68d1b3 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 15 May 2025 19:11:06 +0800 Subject: [PATCH 038/118] save --- dbms/src/Flash/Mpp/MPPTask.cpp | 1 + dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp | 1 + dbms/src/Interpreters/Context.h | 2 +- dbms/src/Operators/CTE.cpp | 13 ------------- dbms/src/Operators/CTE.h | 12 ++++++++++-- 5 files changed, 13 insertions(+), 16 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 647de9c0a16..7620992e4d4 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 1ccd8848477..b191537c076 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -51,6 +51,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( { size_t partition_id = 0; String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); + exec_context.setQueryIDAndCTEID(query_id_and_cte_id); group_builder.transform([&](auto & builder) { builder.setSinkOp(std::make_unique( diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 1fdcdc8fa4b..fc9ed1735f9 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -103,6 +102,7 @@ using TiFlashSecurityConfigPtr = std::shared_ptr; class MockStorage; class JointThreadInfoJeallocMap; using JointThreadInfoJeallocMapPtr = std::shared_ptr; +class CTEManager; enum class PageStorageRunMode : UInt8; namespace DM diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index b099984e61f..194a0edc651 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -18,7 +18,6 @@ #include #include #include -#include namespace DB { @@ -51,18 +50,6 @@ void CTE::pushBlock(const Block & block) this->blocks.push_back(block); } -template -void CTE::notifyEOFImpl() -{ - std::unique_lock lock(this->rw_lock, std::defer_lock); - if constexpr (has_lock) - lock.lock(); - this->is_eof = true; - - // Just in case someone is in WAITING_FOR_NOTIFY status - this->pipe_cv.notifyAll(); -} - void CTE::registerTask(TaskPtr && task) { { diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 6b42e2c83a9..5bc8c1d6267 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -43,7 +43,16 @@ class CTE : public NotifyFuture void notifyEOFNoLock() { this->notifyEOFImpl(); } template - void notifyEOFImpl(); + void notifyEOFImpl() + { + std::unique_lock lock(this->rw_lock, std::defer_lock); + // if constexpr (has_lock) + // lock.lock(); + this->is_eof = true; + + // Just in case someone is in WAITING_FOR_NOTIFY status + this->pipe_cv.notifyAll(); + } void registerTask(TaskPtr && task) override; @@ -69,7 +78,6 @@ class CTE : public NotifyFuture std::deque waiting_tasks; PipeConditionVariable pipe_cv; - // TODO eof can be set only when execution summary has been sent bool is_eof = false; tipb::SelectResponse resp; From 8943a6c4b717b129b880599c58ded1bd2dd79e53 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 15 May 2025 19:19:46 +0800 Subject: [PATCH 039/118] minor fix --- dbms/src/Operators/CTEReader.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 59c03d94fba..d2bb129d1f2 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -48,6 +48,7 @@ class CTEReader std::lock_guard lock(this->mu); if (this->resp_fetched) return; + this->resp_fetched = true; resp.CopyFrom(this->resp); } From 53e4ff52c5b1cb4c0a4ebb4576be30dcc9e45534 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 16 May 2025 11:44:46 +0800 Subject: [PATCH 040/118] fix bug --- dbms/src/Operators/CTE.h | 10 +++++++++- dbms/src/Operators/CTEReader.cpp | 2 +- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 5bc8c1d6267..eac8910fb95 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -63,7 +63,14 @@ class CTE : public NotifyFuture this->notifyEOFNoLock(); } - tipb::SelectResponse getResp() const noexcept { return this->resp; } + void tryToGetResp(tipb::SelectResponse & resp) + { + if (!this->get_resp) + { + this->get_resp = true; + resp.CopyFrom(this->resp); + } + } private: // Return true if CTE has data @@ -80,6 +87,7 @@ class CTE : public NotifyFuture bool is_eof = false; + bool get_resp = false; tipb::SelectResponse resp; }; } // namespace DB diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index e7680bcade5..ffa0a59d0ae 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -34,7 +34,7 @@ std::pair CTEReader::fetchNextBlock() { case FetchStatus::Eof: if (this->resp.execution_summaries_size() == 0) - this->resp = this->cte->getResp(); + this->cte->tryToGetResp(this->resp); case FetchStatus::Waiting: case FetchStatus::Cancelled: return {ret, Block()}; From 83962076e8d28bd2675a87efc9dae27f5d20859e Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 27 May 2025 09:46:42 +0800 Subject: [PATCH 041/118] save --- dbms/src/Flash/Mpp/MPPTask.cpp | 5 +---- dbms/src/Operators/CTESource.cpp | 1 + 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 7620992e4d4..ed9219116df 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -196,10 +196,7 @@ void MPPTask::finishWrite() { const String & query_id_and_cte_id = this->dag_context->getQueryIDAndCTEID(); if (dag_context->collect_execution_summaries) - { - const tipb::SelectResponse & resp = mpp_task_statistics.genExecutionSummaryResponse(); - this->context->getCTEManager()->setRespAndNotifyEOF(resp, query_id_and_cte_id); - } + this->context->getCTEManager()->setRespAndNotifyEOF(mpp_task_statistics.genExecutionSummaryResponse(), query_id_and_cte_id); else this->context->getCTEManager()->notifyEOF(query_id_and_cte_id); } diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index 671c52d5fc2..a819fdda473 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -34,6 +34,7 @@ OperatorStatus CTESourceOp::readImpl(Block & block) this->io_profile_info->remote_execution_summary.add(this->resp); case FetchStatus::Ok: block = res.second; + this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; case FetchStatus::Waiting: if likely (this->cte_reader->isBlockGenerated()) From ac9379726cce13010a20e1e7c0ab3a4bd859beb5 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 27 May 2025 11:02:25 +0800 Subject: [PATCH 042/118] remove useless debug --- dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp index dba6214adc5..a3065b9d5d9 100644 --- a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp @@ -134,19 +134,9 @@ OperatorStatus PipelineExec::executeImpl() assert(!io_op); assert(!waiting_for_notify); - bool do_log = false; - if (sink_op->getName() == "CTESinkOp") - do_log = true; - - auto * log = &Poco::Logger::get("LRUCache"); - Block block; - if (do_log) - LOG_INFO(log, fmt::format("xzxdebug executeImpl: beginning {}", block.dumpStructure())); size_t start_transform_op_index = 0; auto op_status = fetchBlock(block, start_transform_op_index); - if (do_log) - LOG_INFO(log, fmt::format("xzxdebug executeImpl: after fetch {}", block.dumpStructure())); // If the status `fetchBlock` returns isn't `HAS_OUTPUT`, it means that `fetchBlock` did not return a block. if (op_status != OperatorStatus::HAS_OUTPUT) return op_status; @@ -160,10 +150,6 @@ OperatorStatus PipelineExec::executeImpl() { const auto & transform_op = transform_ops[transform_op_index]; op_status = transform_op->transform(block); - if (do_log) - LOG_INFO( - log, - fmt::format("xzxdebug executeImpl: after transform {} {}", transform_op_index, block.dumpStructure())); HANDLE_OP_STATUS(transform_op, op_status, OperatorStatus::HAS_OUTPUT); if (block && block.rows() == 0) return OperatorStatus::NEED_INPUT; From 78f450a26972780288130ab05f5cefeafbf0b7ed Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 28 May 2025 19:02:30 +0800 Subject: [PATCH 043/118] fix bugs --- dbms/src/Flash/Coprocessor/DAGContext.h | 7 ++ .../Flash/Executor/PipelineExecutorContext.h | 5 + dbms/src/Flash/Mpp/CTEManager.cpp | 96 +++++++++++-------- dbms/src/Flash/Mpp/CTEManager.h | 47 ++++++--- dbms/src/Flash/Mpp/MPPTask.cpp | 39 +++++++- dbms/src/Flash/Mpp/MPPTask.h | 3 + .../Flash/Pipeline/Schedule/Tasks/Task.cpp | 5 + dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 3 +- dbms/src/Flash/Planner/PhysicalPlan.cpp | 2 +- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 26 +++-- .../src/Flash/Planner/Plans/PhysicalCTESink.h | 10 +- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 7 +- .../Flash/Planner/Plans/PhysicalCTESource.h | 8 +- dbms/src/Operators/CTE.cpp | 2 + dbms/src/Operators/CTE.h | 46 +++++---- dbms/src/Operators/CTEReader.cpp | 16 ++++ dbms/src/Operators/CTEReader.h | 31 +++++- dbms/src/Operators/CTESinkOp.h | 18 +--- dbms/src/Operators/SharedQueue.cpp | 1 + 19 files changed, 267 insertions(+), 105 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index fbb1650ccfc..ee7ce2b5d1d 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -38,6 +38,7 @@ #include #include #include +#include #include #include #include @@ -368,6 +369,10 @@ class DAGContext this->query_id_and_cte_id = query_id_and_cte_id; } + void sinkNeedRelease() { this->sink_need_release = true; } + std::vector> getCTEs() const { return this->ctes; } + void addCTE(std::shared_ptr & cte) { this->ctes.push_back(cte); } + public: DAGRequest dag_request; /// Some existing code inherited from Clickhouse assume that each query must have a valid query string and query ast, @@ -486,6 +491,8 @@ class DAGContext String connection_alias; String query_id_and_cte_id; + bool sink_need_release = false; + std::vector> ctes; }; } // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.h b/dbms/src/Flash/Executor/PipelineExecutorContext.h index 736ba937c77..dae81033192 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.h +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.h @@ -149,6 +149,11 @@ class PipelineExecutorContext : private boost::noncopyable this->dag_context->setQueryIDAndCTEID(query_id_and_cte_id); } + void addCTE(std::shared_ptr cte) { this->dag_context->addCTE(cte); } + void addCTE(std::shared_ptr & cte) { this->dag_context->addCTE(cte); } + + void sinkNeedRelease() { this->dag_context->sinkNeedRelease(); } + private: bool setExceptionPtr(const std::exception_ptr & exception_ptr_); diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index 60635c5d75b..51e48e15301 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -16,33 +16,13 @@ #include #include +#include #include #include namespace DB { -std::shared_ptr CTEManager::getCTE(const String & query_id_and_cte_id, const String & partition_id) -{ - std::lock_guard lock(this->mu); - auto iter = this->ctes.find(query_id_and_cte_id); - if (iter == this->ctes.end()) - this->ctes[query_id_and_cte_id] = std::map{}; - - auto iter_for_cte = this->ctes[query_id_and_cte_id].find(partition_id); - if (iter_for_cte == this->ctes[query_id_and_cte_id].end()) - { - // It's the first time we request for the specific cte - // Create it because no one created it before. - auto cte = std::make_shared(); - this->ctes[query_id_and_cte_id].insert(std::make_pair(partition_id, CTEWithCounter(cte, 1))); - return cte; - } - - ++(iter_for_cte->second.counter); - return iter_for_cte->second.cte; -} - -void CTEManager::releaseCTE(const String & query_id_and_cte_id, const String & partition_id) +void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id, const String & partition_id) { std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); @@ -51,39 +31,73 @@ void CTEManager::releaseCTE(const String & query_id_and_cte_id, const String & p auto iter_for_cte = iter->second.find(partition_id); if unlikely (iter_for_cte == iter->second.end()) throw Exception(fmt::format("Can't find cte: {}, partition: {}", query_id_and_cte_id, partition_id)); - --(iter_for_cte->second.counter); - if (iter_for_cte->second.counter == 0) + + iter_for_cte->second.sourceExit(); + if (iter_for_cte->second.getTotalExitNum() == iter_for_cte->second.getExpectedTotalNum()) iter->second.erase(iter_for_cte); if (iter->second.size() == 0) + { + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, "xzxdebug releaseCTEBySource, erase {}", query_id_and_cte_id); this->ctes.erase(iter); + } } -void CTEManager::setRespAndNotifyEOF(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) +// TODO refine codes here, do not directly use map +void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) { - this->executeOnManyCTEs(resp, query_id_and_cte_id); -} + std::unique_lock lock(this->mu); + auto iter = this->ctes.find(query_id_and_cte_id); + if unlikely (iter == this->ctes.end()) + throw Exception(fmt::format("Can't find cte: {}", query_id_and_cte_id)); -void CTEManager::notifyEOF(const String & query_id_and_cte_id) -{ - this->executeOnManyCTEs(tipb::SelectResponse(), query_id_and_cte_id); + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, "xzxdebug releaseCTEBySink, counter"); + + auto iter_for_cte = iter->second.begin(); + auto iter_for_cte_end = iter->second.end(); + std::vector ctes_need_erase; + while (iter_for_cte != iter_for_cte_end) + { + CTEWithCounter & cte_with_counter = iter_for_cte->second; + cte_with_counter.getCTE()->addRespAndNotifyEOF(resp); + cte_with_counter.sinkExit(); + if (cte_with_counter.getSinkExitNum() == cte_with_counter.getExpectedSinkNum()) + cte_with_counter.getCTE()->notifyEOF(); + if (cte_with_counter.getTotalExitNum() == cte_with_counter.getExpectedTotalNum()) + ctes_need_erase.push_back(iter_for_cte->first); + iter_for_cte++; + } + + if (ctes_need_erase.size() == iter->second.size()) + this->ctes.erase(iter); + + for (const auto & key : ctes_need_erase) + { + auto iter_for_cte = iter->second.find(key); + iter->second.erase(iter_for_cte); + } } -template -void CTEManager::executeOnManyCTEs(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) +std::shared_ptr CTEManager::getCTEimpl(const String & query_id_and_cte_id, const String & partition_id, Int32 expected_sink_num, Int32 expected_source_num) { std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); - if unlikely (iter == this->ctes.end()) - throw Exception(fmt::format("Can't find cte: {}", query_id_and_cte_id)); - auto iter_for_cte = iter->second.begin(); - while (iter_for_cte != iter->second.end()) + if (iter == this->ctes.end()) + this->ctes[query_id_and_cte_id] = std::map{}; + + auto iter_for_cte = this->ctes[query_id_and_cte_id].find(partition_id); + if (iter_for_cte == this->ctes[query_id_and_cte_id].end()) { - if constexpr (set_resp) - iter_for_cte->second.cte->setRespAndNotifyEOF(resp); - else - iter_for_cte->second.cte->notifyEOF(); - iter_for_cte++; + // It's the first time we request for the specific cte + // Create it because no one created it before. + auto cte = std::make_shared(); + CTEWithCounter cte_with_counter(cte, expected_sink_num, expected_source_num); + this->ctes[query_id_and_cte_id].insert(std::make_pair(partition_id, cte_with_counter)); + return cte; } + + return iter_for_cte->second.getCTE(); } } // namespace DB diff --git a/dbms/src/Flash/Mpp/CTEManager.h b/dbms/src/Flash/Mpp/CTEManager.h index 1a6a0d5ba08..6f019c252dc 100644 --- a/dbms/src/Flash/Mpp/CTEManager.h +++ b/dbms/src/Flash/Mpp/CTEManager.h @@ -16,33 +16,58 @@ #include +#include #include #include namespace DB { -struct CTEWithCounter +// Data race is prevented by the lock in CTEManager +class CTEWithCounter { - CTEWithCounter(std::shared_ptr cte_, Int32 counter_) - : cte(cte_) - , counter(counter_) +public: + explicit CTEWithCounter(std::shared_ptr cte_, Int32 expected_total_sink_num_, Int32 expected_total_source_num_) + : cte(cte_), expected_total_sink_num(expected_total_sink_num_), expected_total_source_num(expected_total_source_num_) {} + + void sinkExit() { this->sink_exit_num++; } + void sourceExit() { this->source_exit_num++; } + + Int32 getSinkExitNum() const { return this->sink_exit_num; } + Int32 getSourceExitNum() const { return this->source_exit_num; } + Int32 getTotalExitNum() const { return this->getSinkExitNum() + this->getSourceExitNum(); } + + Int32 getExpectedSinkNum() const { return this->expected_total_sink_num; } + Int32 getExpectedTotalNum() const { return this->getExpectedSinkNum() + this->expected_total_source_num; } + + std::shared_ptr getCTE() const { return this->cte; } + +private: std::shared_ptr cte; - Int32 counter; + + Int32 sink_exit_num = 0; + Int32 source_exit_num = 0; + Int32 expected_total_sink_num; + Int32 expected_total_source_num; }; // TODO Test this class with UT class CTEManager { public: - std::shared_ptr getCTE(const String & query_id_and_cte_id, const String & partition_id); - void releaseCTE(const String & query_id_and_cte_id, const String & partition_id); - void setRespAndNotifyEOF(const tipb::SelectResponse & resp, const String & query_id_and_cte_id); - void notifyEOF(const String & query_id_and_cte_id); + std::shared_ptr getCTEBySink(const String & query_id_and_cte_id, const String & partition_id, Int32 expected_sink_num, Int32 expected_source_num) + { + return this->getCTEimpl(query_id_and_cte_id, partition_id, expected_sink_num, expected_source_num); + } + std::shared_ptr getCTEBySource(const String & query_id_and_cte_id, const String & partition_id, Int32 expected_sink_num, Int32 expected_source_num) + { + return this->getCTEimpl(query_id_and_cte_id, partition_id, expected_sink_num, expected_source_num); + } + void releaseCTEBySource(const String & query_id_and_cte_id, const String & partition_id); + void releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id); private: - template - void executeOnManyCTEs(const tipb::SelectResponse & resp, const String & query_id_and_cte_id); + std::shared_ptr getCTEimpl(const String & query_id_and_cte_id, const String & partition_id, Int32 expected_sink_num, Int32 expected_source_num); std::mutex mu; std::map> ctes; diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index ed9219116df..265c01ded7b 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -190,15 +190,38 @@ void MPPTask::abortQueryExecutor() } } +void MPPTask::abortCTE() +{ + if (this->has_cte_source) + { + auto ctes = this->dag_context->getCTEs(); + + // CTESource may be waiting for the eof from cte sink + // We'd better to manually do eof notification in case of missing eof from cte sink + // or the CTESource will hang + for (auto & cte : ctes) + cte->notifyEOF(); + } + + // TODO always delete cte when we receive cancel +} + void MPPTask::finishWrite() { if (this->has_cte_sink) { const String & query_id_and_cte_id = this->dag_context->getQueryIDAndCTEID(); + LOG_INFO(log, "xzxdebug enter finishWrite, query_id_and_cte_id: {}, mpptask id: {}", query_id_and_cte_id, id.toString()); + CTEManager * cte_manager = this->context->getCTEManager(); + tipb::SelectResponse resp; if (dag_context->collect_execution_summaries) - this->context->getCTEManager()->setRespAndNotifyEOF(mpp_task_statistics.genExecutionSummaryResponse(), query_id_and_cte_id); - else - this->context->getCTEManager()->notifyEOF(query_id_and_cte_id); + resp = mpp_task_statistics.genExecutionSummaryResponse(); + + // The finish of pushing all blocks not means that cte sink job has been done. + // Execution summary statistic also need to be sent. So we can release cte + // only when execution sumary statistic has been sent. + cte_manager->releaseCTEBySink(resp, query_id_and_cte_id); + this->notify_cte_eof = true; } else { @@ -222,6 +245,8 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) if unlikely (!dag_context->dag_request.rootExecutor().has_cte_sink()) throw Exception("Task has either exchange sender or cte sink"); + LOG_INFO(log, "xzxdebug register cte sink"); + // There is no need to register tunnel for cte sink this->has_cte_sink = true; return; @@ -561,7 +586,9 @@ void MPPTask::runImpl() dag_context->tunnel_set->getExternalThreadCnt(), new_thread_count_of_mpp_receiver); + LOG_INFO(log, "xzxdebug wait for schedule"); scheduleOrWait(); + LOG_INFO(log, "xzxdebug start to run"); auto time_cost_in_schedule_ns = stopwatch.elapsed() - time_cost_in_preprocess_ns; dag_context->minTSO_wait_time_ns = time_cost_in_schedule_ns; @@ -593,6 +620,7 @@ void MPPTask::runImpl() #endif auto result = query_executor_holder->execute(); + LOG_INFO(log, "xzxdebug execution is finished"); auto log_level = Poco::Message::PRIO_DEBUG; if (!result.is_success || status != RUNNING) log_level = Poco::Message::PRIO_INFORMATION; @@ -647,6 +675,11 @@ void MPPTask::runImpl() err_msg = err_msg.empty() ? catch_err_msg : fmt::format("{}, {}", err_msg, catch_err_msg); } + if (this->has_cte_sink && !this->notify_cte_eof) + { + // TODO + } + if (err_msg.empty()) { reportStatus(""); diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index db9330f2c0a..80204001d47 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -124,6 +124,7 @@ class MPPTask void abortTunnels(const String & message, bool wait_sender_finish); void abortReceivers(); void abortQueryExecutor(); + void abortCTE(); void finishWrite(); @@ -200,6 +201,8 @@ class MPPTask MPPReceiverSetPtr receiver_set; bool has_cte_sink = false; + bool has_cte_source = false; + bool notify_cte_eof = false; int new_thread_count_of_mpp_receiver = 0; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp index 86ca4b2b0eb..3f4058c55b1 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp @@ -120,6 +120,11 @@ Task::~Task() return task_status; \ } +void Task::onErrorOccurred(const String & err_msg) +{ + exec_context.onErrorOccurred(err_msg); +} + ExecTaskStatus Task::execute() { assert(current_notify_future == nullptr); diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h index 46670798b5c..809fdc2cfff 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -16,7 +16,6 @@ #include #include -#include #include #include #include @@ -117,7 +116,7 @@ class Task const PipelineExecutorContext & getQueryExecContext() { return exec_context; } - void onErrorOccurred(const String & err_msg) { exec_context.onErrorOccurred(err_msg); } + void onErrorOccurred(const String & err_msg); void setNotifyType(NotifyType type) { notify_type = type; } NotifyType getNotifyType() const { return notify_type; } diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 877e4725e70..df6f3838d1f 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -254,7 +254,7 @@ void PhysicalPlan::build(const tipb::Executor * executor) auto fine_grained_shuffle = FineGrainedShuffle(executor); GET_METRIC(tiflash_coprocessor_executor_count, type_cte_sink).Increment(); pushBack( - PhysicalCTESink::build(executor_id, log, fine_grained_shuffle, popBack(), executor->cte_sink().cte_id())); + PhysicalCTESink::build(executor_id, log, fine_grained_shuffle, popBack(), executor->cte_sink())); break; } default: diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index b191537c076..83cbebdfe98 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -28,7 +29,7 @@ PhysicalPlanNodePtr PhysicalCTESink::build( const LoggerPtr & log, const FineGrainedShuffle & fine_grained_shuffle, const PhysicalPlanNodePtr & child, - UInt32 cte_id) + const ::tipb::CTESink & cte_sink) { RUNTIME_CHECK(child); @@ -38,7 +39,9 @@ PhysicalPlanNodePtr PhysicalCTESink::build( fine_grained_shuffle, log->identifier(), child, - cte_id); + cte_sink.cte_id(), + cte_sink.cte_sink_num(), + cte_sink.cte_source_num()); physical_cte_sink->disableRestoreConcurrency(); return physical_cte_sink; } @@ -52,15 +55,26 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( size_t partition_id = 0; String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); exec_context.setQueryIDAndCTEID(query_id_and_cte_id); + + std::shared_ptr cte; + if (!fine_grained_shuffle.enabled()) + { + cte = context.getCTEManager()->getCTEBySink(query_id_and_cte_id, "", this->expected_sink_num, this->expected_source_num); + exec_context.sinkNeedRelease(); + } + group_builder.transform([&](auto & builder) { + if (fine_grained_shuffle.enabled()) + { + cte = context.getCTEManager()->getCTEBySink(query_id_and_cte_id, std::to_string(partition_id), this->expected_sink_num, this->expected_source_num); + partition_id++; + } + builder.setSinkOp(std::make_unique( exec_context, log->identifier(), - query_id_and_cte_id, - fine_grained_shuffle.enabled() ? std::to_string(partition_id) : "", - context.getCTEManager())); - partition_id++; + cte)); }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h index 7f22406f97b..76126975d39 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h @@ -29,7 +29,7 @@ class PhysicalCTESink : public PhysicalUnary const LoggerPtr & log, const FineGrainedShuffle & fine_grained_shuffle, const PhysicalPlanNodePtr & child, - UInt32 cte_id); + const ::tipb::CTESink & cte_sink); PhysicalCTESink( const String & executor_id_, @@ -37,9 +37,13 @@ class PhysicalCTESink : public PhysicalUnary const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, - UInt32 cte_id_) + UInt32 cte_id_, + Int32 expected_sink_num_, + Int32 expected_source_num_) : PhysicalUnary(executor_id_, PlanType::CTESink, schema_, fine_grained_shuffle_, req_id, child_) , cte_id(cte_id_) + , expected_sink_num(expected_sink_num_) + , expected_source_num(expected_source_num_) {} void finalizeImpl(const Names & parent_require) override; @@ -54,5 +58,7 @@ class PhysicalCTESink : public PhysicalUnary size_t /*concurrency*/) override; UInt32 cte_id; + Int32 expected_sink_num; + Int32 expected_source_num; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 4cd487d6848..cbfb053b99b 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -66,16 +66,19 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( { for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) { + auto cte_reader = std::make_shared(query_id_and_cte_id, std::to_string(partition_id), context.getCTEManager(), this->expected_sink_num, this->expected_source_num); + exec_context.addCTE(cte_reader->getCTE()); group_builder.addConcurrency(std::make_unique( exec_context, log->identifier(), - std::make_shared(query_id_and_cte_id, std::to_string(partition_id), context.getCTEManager()), + cte_reader, schema)); } } else { - auto cte_reader = std::make_shared(query_id_and_cte_id, "", context.getCTEManager()); + auto cte_reader = std::make_shared(query_id_and_cte_id, "", context.getCTEManager(), this->expected_sink_num, this->expected_source_num); + exec_context.addCTE(cte_reader->getCTE()); for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) { group_builder.addConcurrency( diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h index 493ab1bf46a..e2e17d29ce2 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -37,10 +37,14 @@ class PhysicalCTESource : public PhysicalLeaf const FineGrainedShuffle & fine_grained_shuffle, const String & req_id, const Block & sample_block_, - UInt64 cte_id_) + UInt64 cte_id_, + Int32 expected_sink_num_, + Int32 expected_source_num_) : PhysicalLeaf(executor_id_, PlanType::CTESource, schema_, fine_grained_shuffle, req_id) , sample_block(sample_block_) , cte_id(cte_id_) + , expected_sink_num(expected_sink_num_) + , expected_source_num(expected_source_num_) {} void finalizeImpl(const Names & parent_require) override; @@ -60,5 +64,7 @@ class PhysicalCTESource : public PhysicalLeaf UInt64 cte_id; std::vector partition_col_ids; TiDB::TiDBCollators partition_col_collators; + Int32 expected_sink_num; + Int32 expected_source_num; }; } // namespace DB diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 194a0edc651..b8bb1a3fdb3 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -48,6 +48,8 @@ void CTE::pushBlock(const Block & block) if unlikely (this->blocks.empty()) this->pipe_cv.notifyAll(); this->blocks.push_back(block); + this->block_num++; + this->row_num += block.rows(); } void CTE::registerTask(TaskPtr && task) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index eac8910fb95..f9667ac888f 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -20,6 +20,7 @@ #include #include +#include #include namespace DB @@ -35,31 +36,22 @@ enum class FetchStatus class CTE : public NotifyFuture { public: - ~CTE() override = default; + ~CTE() override + { + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, fmt::format("xzxdebug block num: {}, row num: {}", this->block_num, this->row_num)); + } FetchStatus tryGetBunchBlocks(size_t idx, std::deque & queue); void pushBlock(const Block & block); void notifyEOF() { this->notifyEOFImpl(); } - void notifyEOFNoLock() { this->notifyEOFImpl(); } - - template - void notifyEOFImpl() - { - std::unique_lock lock(this->rw_lock, std::defer_lock); - // if constexpr (has_lock) - // lock.lock(); - this->is_eof = true; - - // Just in case someone is in WAITING_FOR_NOTIFY status - this->pipe_cv.notifyAll(); - } void registerTask(TaskPtr && task) override; - void setRespAndNotifyEOF(const tipb::SelectResponse & resp) + void addRespAndNotifyEOF(const tipb::SelectResponse & resp) { std::unique_lock lock(this->rw_lock); - this->resp = resp; + this->resp.MergeFrom(resp); this->notifyEOFNoLock(); } @@ -72,7 +64,26 @@ class CTE : public NotifyFuture } } + Int64 blockNumForTest() { + std::unique_lock lock(this->rw_lock); + return this->blocks.size(); + } + private: + void notifyEOFNoLock() { this->notifyEOFImpl(); } + + template + void notifyEOFImpl() + { + std::unique_lock lock(this->rw_lock, std::defer_lock); + if constexpr (has_lock) + lock.lock(); + this->is_eof = true; + + // Just in case someone is in WAITING_FOR_NOTIFY status + this->pipe_cv.notifyAll(); + } + // Return true if CTE has data inline bool hasDataNoLock() const { return !this->blocks.empty(); } @@ -89,5 +100,8 @@ class CTE : public NotifyFuture bool get_resp = false; tipb::SelectResponse resp; + + Int64 block_num = 0; + Int64 row_num = 0; }; } // namespace DB diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index ffa0a59d0ae..71ffb7ec74b 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -26,10 +26,17 @@ std::pair CTEReader::fetchNextBlock() { Block block = std::move(this->blocks.front()); this->blocks.pop_front(); + this->output_block_num++; + this->output_row_num += block.rows(); return {FetchStatus::Ok, block}; } auto ret = this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); + // if (ret != FetchStatus::Ok) + // { + // auto * log = &Poco::Logger::get("LRUCache"); + // LOG_INFO(log, "xzxdebug block_fetch_idx: {}, block num in cte: {}", this->block_fetch_idx, this->cte->blockNumForTest()); + // } switch (ret) { case FetchStatus::Eof: @@ -39,9 +46,12 @@ std::pair CTEReader::fetchNextBlock() case FetchStatus::Cancelled: return {ret, Block()}; case FetchStatus::Ok: + this->save_block_num += this->blocks.size(); this->block_fetch_idx += this->blocks.size(); Block block = std::move(this->blocks.front()); this->blocks.pop_front(); + this->output_block_num++; + this->output_row_num += block.rows(); return {ret, block}; } throw Exception("Should not reach here"); @@ -54,6 +64,11 @@ FetchStatus CTEReader::checkAvailableBlock() return FetchStatus::Ok; auto ret = this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); + // if (ret != FetchStatus::Ok) + // { + // auto * log = &Poco::Logger::get("LRUCache"); + // LOG_INFO(log, "xzxdebug block_fetch_idx: {}, block num in cte: {}", this->block_fetch_idx, this->cte->blockNumForTest()); + // } switch (ret) { case FetchStatus::Waiting: @@ -61,6 +76,7 @@ FetchStatus CTEReader::checkAvailableBlock() case FetchStatus::Eof: return ret; case FetchStatus::Ok: + this->save_block_num += this->blocks.size(); this->block_fetch_idx += this->blocks.size(); return FetchStatus::Ok; } diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index d2bb129d1f2..2627973dbdd 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -20,24 +20,40 @@ #include #include +#include #include +// TODO remove them +#include +#include + namespace DB { class CTEReader { public: - CTEReader(const String & query_id_and_cte_id_, const String & partition_id_, CTEManager * cte_manager_) + CTEReader(const String & query_id_and_cte_id_, const String & partition_id_, CTEManager * cte_manager_, Int32 expected_sink_num_, Int32 expected_source_num_) : query_id_and_cte_id(query_id_and_cte_id_) , partition_id(partition_id_) , cte_manager(cte_manager_) - , cte(cte_manager_->getCTE(query_id_and_cte_id_, partition_id)) + , cte(cte_manager_->getCTEBySource(query_id_and_cte_id_, partition_id, expected_sink_num_, expected_source_num_)) {} ~CTEReader() { this->cte.reset(); - this->cte_manager->releaseCTE(this->query_id_and_cte_id, this->partition_id); + this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id, this->partition_id); + + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO( + log, + fmt::format( + "xzxdebug query_id: {}, pid: {}, output block num: {}, output row num: {}, save block num:{}", + this->query_id_and_cte_id, + this->partition_id, + this->output_block_num, + this->output_row_num, + this->save_block_num)); } std::pair fetchNextBlock(); @@ -57,13 +73,20 @@ class CTEReader std::lock_guard lock(this->mu); // `block_fetch_idx == 0` means that CTE hasn't received block yet, maybe it is waiting - // for the finish of join executor + // for the finish of join executor and etc. return this->block_fetch_idx != 0; } void setNotifyFuture() { ::DB::setNotifyFuture(cte.get()); } + std::shared_ptr getCTE() const { return this->cte; } + private: + Int64 output_block_num = 0; + Int64 output_row_num = 0; + + Int64 save_block_num = 0; + String query_id_and_cte_id; String partition_id; CTEManager * cte_manager; diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 0766d93f505..c17827dc526 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -26,33 +26,19 @@ class CTESinkOp : public SinkOp CTESinkOp( PipelineExecutorContext & exec_context_, const String & req_id, - const String & query_id_and_cte_id_, - const String & partition_id_, - CTEManager * cte_manager_) + std::shared_ptr cte_) : SinkOp(exec_context_, req_id) - , query_id_and_cte_id(query_id_and_cte_id_) - , cte_manager(cte_manager_) - , cte(cte_manager_->getCTE(query_id_and_cte_id_, partition_id_)) + , cte(cte_) {} - ~CTESinkOp() override - { - this->cte.reset(); - this->cte_manager->releaseCTE(this->query_id_and_cte_id, this->partition_id); - } - String getName() const override { return "CTESinkOp"; } bool canHandleSelectiveBlock() const override { return true; } - protected: void operateSuffixImpl() override; OperatorStatus writeImpl(Block && block) override; private: - String query_id_and_cte_id; - String partition_id; - CTEManager * cte_manager; std::shared_ptr cte; size_t total_rows = 0; bool input_done = false; diff --git a/dbms/src/Operators/SharedQueue.cpp b/dbms/src/Operators/SharedQueue.cpp index 12ba634d8c8..8dc024ca72f 100644 --- a/dbms/src/Operators/SharedQueue.cpp +++ b/dbms/src/Operators/SharedQueue.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include From f6ee482d26028ba0e8a13fb85f38e7535ae71a00 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 29 May 2025 11:02:08 +0800 Subject: [PATCH 044/118] fix bugs --- dbms/src/Flash/Coprocessor/DAGContext.h | 5 ++- .../Flash/Executor/PipelineExecutorContext.h | 3 +- dbms/src/Flash/Mpp/CTEManager.cpp | 22 +++++++++++-- dbms/src/Flash/Mpp/CTEManager.h | 23 ++++++++++--- dbms/src/Flash/Mpp/MPPTask.cpp | 32 +++++++++++-------- dbms/src/Flash/Mpp/MPPTask.h | 2 +- dbms/src/Flash/Mpp/MPPTaskId.h | 5 +-- dbms/src/Flash/Planner/PhysicalPlan.cpp | 3 +- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 16 ++++++---- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 23 +++++++++---- dbms/src/Operators/CTE.cpp | 8 ++++- dbms/src/Operators/CTE.h | 19 +++++++---- dbms/src/Operators/CTEReader.h | 10 ++++-- dbms/src/Operators/CTESinkOp.cpp | 5 +-- dbms/src/Operators/CTESinkOp.h | 5 +-- dbms/src/Operators/SharedQueue.cpp | 2 +- 16 files changed, 125 insertions(+), 58 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index ee7ce2b5d1d..7f6f98447ad 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -36,9 +36,9 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -372,6 +372,8 @@ class DAGContext void sinkNeedRelease() { this->sink_need_release = true; } std::vector> getCTEs() const { return this->ctes; } void addCTE(std::shared_ptr & cte) { this->ctes.push_back(cte); } + bool hasCTESource() const { return this->has_cte_source; } + void setHasCTESource() { this->has_cte_source = true; } public: DAGRequest dag_request; @@ -491,6 +493,7 @@ class DAGContext String connection_alias; String query_id_and_cte_id; + bool has_cte_source = false; bool sink_need_release = false; std::vector> ctes; }; diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.h b/dbms/src/Flash/Executor/PipelineExecutorContext.h index dae81033192..2d8ee77f5ee 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.h +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.h @@ -153,7 +153,8 @@ class PipelineExecutorContext : private boost::noncopyable void addCTE(std::shared_ptr & cte) { this->dag_context->addCTE(cte); } void sinkNeedRelease() { this->dag_context->sinkNeedRelease(); } - + void setHasCTESource() { this->dag_context->setHasCTESource(); } + private: bool setExceptionPtr(const std::exception_ptr & exception_ptr_); diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index 51e48e15301..eb922a6c447 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -31,7 +31,7 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id, const St auto iter_for_cte = iter->second.find(partition_id); if unlikely (iter_for_cte == iter->second.end()) throw Exception(fmt::format("Can't find cte: {}, partition: {}", query_id_and_cte_id, partition_id)); - + iter_for_cte->second.sourceExit(); if (iter_for_cte->second.getTotalExitNum() == iter_for_cte->second.getExpectedTotalNum()) iter->second.erase(iter_for_cte); @@ -50,7 +50,8 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin std::unique_lock lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if unlikely (iter == this->ctes.end()) - throw Exception(fmt::format("Can't find cte: {}", query_id_and_cte_id)); + // Maybe the task is cancelled and all ctes have been released + return; auto * log = &Poco::Logger::get("LRUCache"); LOG_INFO(log, "xzxdebug releaseCTEBySink, counter"); @@ -71,7 +72,10 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin } if (ctes_need_erase.size() == iter->second.size()) + { this->ctes.erase(iter); + return; + } for (const auto & key : ctes_need_erase) { @@ -80,7 +84,19 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin } } -std::shared_ptr CTEManager::getCTEimpl(const String & query_id_and_cte_id, const String & partition_id, Int32 expected_sink_num, Int32 expected_source_num) +void CTEManager::releaseCTEs(const String & query_id_and_cte_id) +{ + std::lock_guard lock(this->mu); + auto iter = this->ctes.find(query_id_and_cte_id); + if (iter != this->ctes.end()) + this->ctes.erase(iter); +} + +std::shared_ptr CTEManager::getCTEimpl( + const String & query_id_and_cte_id, + const String & partition_id, + Int32 expected_sink_num, + Int32 expected_source_num) { std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); diff --git a/dbms/src/Flash/Mpp/CTEManager.h b/dbms/src/Flash/Mpp/CTEManager.h index 6f019c252dc..5d4e544e373 100644 --- a/dbms/src/Flash/Mpp/CTEManager.h +++ b/dbms/src/Flash/Mpp/CTEManager.h @@ -27,7 +27,9 @@ class CTEWithCounter { public: explicit CTEWithCounter(std::shared_ptr cte_, Int32 expected_total_sink_num_, Int32 expected_total_source_num_) - : cte(cte_), expected_total_sink_num(expected_total_sink_num_), expected_total_source_num(expected_total_source_num_) + : cte(cte_) + , expected_total_sink_num(expected_total_sink_num_) + , expected_total_source_num(expected_total_source_num_) {} void sinkExit() { this->sink_exit_num++; } @@ -55,19 +57,32 @@ class CTEWithCounter class CTEManager { public: - std::shared_ptr getCTEBySink(const String & query_id_and_cte_id, const String & partition_id, Int32 expected_sink_num, Int32 expected_source_num) + std::shared_ptr getCTEBySink( + const String & query_id_and_cte_id, + const String & partition_id, + Int32 expected_sink_num, + Int32 expected_source_num) { return this->getCTEimpl(query_id_and_cte_id, partition_id, expected_sink_num, expected_source_num); } - std::shared_ptr getCTEBySource(const String & query_id_and_cte_id, const String & partition_id, Int32 expected_sink_num, Int32 expected_source_num) + std::shared_ptr getCTEBySource( + const String & query_id_and_cte_id, + const String & partition_id, + Int32 expected_sink_num, + Int32 expected_source_num) { return this->getCTEimpl(query_id_and_cte_id, partition_id, expected_sink_num, expected_source_num); } void releaseCTEBySource(const String & query_id_and_cte_id, const String & partition_id); void releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id); + void releaseCTEs(const String & query_id_and_cte_id); private: - std::shared_ptr getCTEimpl(const String & query_id_and_cte_id, const String & partition_id, Int32 expected_sink_num, Int32 expected_source_num); + std::shared_ptr getCTEimpl( + const String & query_id_and_cte_id, + const String & partition_id, + Int32 expected_sink_num, + Int32 expected_source_num); std::mutex mu; std::map> ctes; diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 265c01ded7b..567c8e87a52 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -191,28 +191,29 @@ void MPPTask::abortQueryExecutor() } void MPPTask::abortCTE() -{ - if (this->has_cte_source) +{ + if (this->dag_context->hasCTESource()) { auto ctes = this->dag_context->getCTEs(); - // CTESource may be waiting for the eof from cte sink - // We'd better to manually do eof notification in case of missing eof from cte sink + // CTESource may be waiting for the finish signal from cte sink + // We'd better to manually do notification in case of missing signal from cte sink // or the CTESource will hang for (auto & cte : ctes) - cte->notifyEOF(); + cte->notifyCancel(); + this->context->getCTEManager()->releaseCTEs(this->dag_context->getQueryIDAndCTEID()); } - - // TODO always delete cte when we receive cancel } void MPPTask::finishWrite() { if (this->has_cte_sink) { - const String & query_id_and_cte_id = this->dag_context->getQueryIDAndCTEID(); - LOG_INFO(log, "xzxdebug enter finishWrite, query_id_and_cte_id: {}, mpptask id: {}", query_id_and_cte_id, id.toString()); - CTEManager * cte_manager = this->context->getCTEManager(); + LOG_INFO( + log, + "xzxdebug enter finishWrite, query_id_and_cte_id: {}, mpptask id: {}", + this->dag_context->getQueryIDAndCTEID(), + id.toString()); tipb::SelectResponse resp; if (dag_context->collect_execution_summaries) resp = mpp_task_statistics.genExecutionSummaryResponse(); @@ -220,8 +221,8 @@ void MPPTask::finishWrite() // The finish of pushing all blocks not means that cte sink job has been done. // Execution summary statistic also need to be sent. So we can release cte // only when execution sumary statistic has been sent. - cte_manager->releaseCTEBySink(resp, query_id_and_cte_id); - this->notify_cte_eof = true; + this->context->getCTEManager()->releaseCTEBySink(resp, this->dag_context->getQueryIDAndCTEID()); + this->notify_cte_finish = true; } else { @@ -675,9 +676,11 @@ void MPPTask::runImpl() err_msg = err_msg.empty() ? catch_err_msg : fmt::format("{}, {}", err_msg, catch_err_msg); } - if (this->has_cte_sink && !this->notify_cte_eof) + if (this->has_cte_sink && !this->notify_cte_finish) { - // TODO + tipb::SelectResponse resp; + this->context->getCTEManager()->releaseCTEBySink(resp, this->dag_context->getQueryIDAndCTEID()); + this->notify_cte_finish = true; } if (err_msg.empty()) @@ -832,6 +835,7 @@ void MPPTask::abort(const String & message, AbortType abort_type) abortTunnels(message, false); abortReceivers(); abortQueryExecutor(); + abortCTE(); scheduleThisTask(ScheduleState::FAILED); /// runImpl is running, leave remaining work to runImpl LOG_WARNING(log, "Finish abort task from running"); diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index 80204001d47..838c8027b24 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -202,7 +202,7 @@ class MPPTask bool has_cte_sink = false; bool has_cte_source = false; - bool notify_cte_eof = false; + bool notify_cte_finish = false; int new_thread_count_of_mpp_receiver = 0; diff --git a/dbms/src/Flash/Mpp/MPPTaskId.h b/dbms/src/Flash/Mpp/MPPTaskId.h index 6005d5b872f..e3789f04795 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.h +++ b/dbms/src/Flash/Mpp/MPPTaskId.h @@ -111,11 +111,12 @@ struct MPPGatherId inline String toStringForQueryID() const { return fmt::format( - "query_ts:{}, local_query_id:{}, server_id:{}, start_ts:{}", + "query_ts:{}, local_query_id:{}, server_id:{}, start_ts:{}, gather_id: {}", query_id.query_ts, query_id.local_query_id, query_id.server_id, - query_id.start_ts); + query_id.start_ts, + this->gather_id); } inline String toString() const { return fmt::format("gather_id:{}, {}", gather_id, this->toStringForQueryID()); } diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index df6f3838d1f..3e38cff56a8 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -253,8 +253,7 @@ void PhysicalPlan::build(const tipb::Executor * executor) buildFinalProjectionForCTE(executor->cte_sink()); auto fine_grained_shuffle = FineGrainedShuffle(executor); GET_METRIC(tiflash_coprocessor_executor_count, type_cte_sink).Increment(); - pushBack( - PhysicalCTESink::build(executor_id, log, fine_grained_shuffle, popBack(), executor->cte_sink())); + pushBack(PhysicalCTESink::build(executor_id, log, fine_grained_shuffle, popBack(), executor->cte_sink())); break; } default: diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 83cbebdfe98..4f636c0a133 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -55,11 +55,12 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( size_t partition_id = 0; String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); exec_context.setQueryIDAndCTEID(query_id_and_cte_id); - + std::shared_ptr cte; if (!fine_grained_shuffle.enabled()) { - cte = context.getCTEManager()->getCTEBySink(query_id_and_cte_id, "", this->expected_sink_num, this->expected_source_num); + cte = context.getCTEManager() + ->getCTEBySink(query_id_and_cte_id, "", this->expected_sink_num, this->expected_source_num); exec_context.sinkNeedRelease(); } @@ -67,14 +68,15 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( group_builder.transform([&](auto & builder) { if (fine_grained_shuffle.enabled()) { - cte = context.getCTEManager()->getCTEBySink(query_id_and_cte_id, std::to_string(partition_id), this->expected_sink_num, this->expected_source_num); + cte = context.getCTEManager()->getCTEBySink( + query_id_and_cte_id, + std::to_string(partition_id), + this->expected_sink_num, + this->expected_source_num); partition_id++; } - builder.setSinkOp(std::make_unique( - exec_context, - log->identifier(), - cte)); + builder.setSinkOp(std::make_unique(exec_context, log->identifier(), cte)); }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index cbfb053b99b..b80a20226b1 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -61,23 +61,32 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); + exec_context.setQueryIDAndCTEID(query_id_and_cte_id); + exec_context.setHasCTESource(); if (fine_grained_shuffle.enabled()) { for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) { - auto cte_reader = std::make_shared(query_id_and_cte_id, std::to_string(partition_id), context.getCTEManager(), this->expected_sink_num, this->expected_source_num); + auto cte_reader = std::make_shared( + query_id_and_cte_id, + std::to_string(partition_id), + context.getCTEManager(), + this->expected_sink_num, + this->expected_source_num); exec_context.addCTE(cte_reader->getCTE()); - group_builder.addConcurrency(std::make_unique( - exec_context, - log->identifier(), - cte_reader, - schema)); + group_builder.addConcurrency( + std::make_unique(exec_context, log->identifier(), cte_reader, schema)); } } else { - auto cte_reader = std::make_shared(query_id_and_cte_id, "", context.getCTEManager(), this->expected_sink_num, this->expected_source_num); + auto cte_reader = std::make_shared( + query_id_and_cte_id, + "", + context.getCTEManager(), + this->expected_sink_num, + this->expected_source_num); exec_context.addCTE(cte_reader->getCTE()); for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) { diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index b8bb1a3fdb3..060783668bf 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -26,6 +26,9 @@ FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) assert(queue.empty()); std::shared_lock lock(this->rw_lock); + if unlikely (this->is_cancelled) + return FetchStatus::Cancelled; + auto block_num = this->blocks.size(); if (block_num <= idx) { @@ -40,9 +43,11 @@ FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) return FetchStatus::Ok; } -void CTE::pushBlock(const Block & block) +bool CTE::pushBlock(const Block & block) { std::unique_lock lock(this->rw_lock); + if unlikely (this->is_cancelled) + return false; this->memory_usage += block.bytes(); if unlikely (this->blocks.empty()) @@ -50,6 +55,7 @@ void CTE::pushBlock(const Block & block) this->blocks.push_back(block); this->block_num++; this->row_num += block.rows(); + return true; } void CTE::registerTask(TaskPtr && task) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index f9667ac888f..38f9214cc92 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -43,8 +43,9 @@ class CTE : public NotifyFuture } FetchStatus tryGetBunchBlocks(size_t idx, std::deque & queue); - void pushBlock(const Block & block); - void notifyEOF() { this->notifyEOFImpl(); } + bool pushBlock(const Block & block); + void notifyEOF() { this->notifyImpl(true); } + void notifyCancel() { this->notifyImpl(false); } void registerTask(TaskPtr && task) override; @@ -64,21 +65,26 @@ class CTE : public NotifyFuture } } - Int64 blockNumForTest() { + Int64 blockNumForTest() + { std::unique_lock lock(this->rw_lock); return this->blocks.size(); } private: - void notifyEOFNoLock() { this->notifyEOFImpl(); } + void notifyEOFNoLock() { this->notifyImpl(true); } template - void notifyEOFImpl() + void notifyImpl(bool is_eof) { std::unique_lock lock(this->rw_lock, std::defer_lock); if constexpr (has_lock) lock.lock(); - this->is_eof = true; + + if likely (is_eof) + this->is_eof = true; + else + this->is_cancelled = true; // Just in case someone is in WAITING_FOR_NOTIFY status this->pipe_cv.notifyAll(); @@ -97,6 +103,7 @@ class CTE : public NotifyFuture PipeConditionVariable pipe_cv; bool is_eof = false; + bool is_cancelled = false; bool get_resp = false; tipb::SelectResponse resp; diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 2627973dbdd..b2d02c3c164 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -32,11 +32,17 @@ namespace DB class CTEReader { public: - CTEReader(const String & query_id_and_cte_id_, const String & partition_id_, CTEManager * cte_manager_, Int32 expected_sink_num_, Int32 expected_source_num_) + CTEReader( + const String & query_id_and_cte_id_, + const String & partition_id_, + CTEManager * cte_manager_, + Int32 expected_sink_num_, + Int32 expected_source_num_) : query_id_and_cte_id(query_id_and_cte_id_) , partition_id(partition_id_) , cte_manager(cte_manager_) - , cte(cte_manager_->getCTEBySource(query_id_and_cte_id_, partition_id, expected_sink_num_, expected_source_num_)) + , cte(cte_manager_ + ->getCTEBySource(query_id_and_cte_id_, partition_id, expected_sink_num_, expected_source_num_)) {} ~CTEReader() diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 331e9b1ce41..71b3e8642f5 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -30,7 +30,8 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) return OperatorStatus::FINISHED; } this->total_rows += block.rows(); - this->cte->pushBlock(block); - return OperatorStatus::NEED_INPUT; + if (this->cte->pushBlock(block)) + return OperatorStatus::NEED_INPUT; + return OperatorStatus::CANCELLED; } } // namespace DB diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index c17827dc526..480b94a5437 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -23,10 +23,7 @@ namespace DB class CTESinkOp : public SinkOp { public: - CTESinkOp( - PipelineExecutorContext & exec_context_, - const String & req_id, - std::shared_ptr cte_) + CTESinkOp(PipelineExecutorContext & exec_context_, const String & req_id, std::shared_ptr cte_) : SinkOp(exec_context_, req_id) , cte(cte_) {} diff --git a/dbms/src/Operators/SharedQueue.cpp b/dbms/src/Operators/SharedQueue.cpp index 8dc024ca72f..cf2a163b52f 100644 --- a/dbms/src/Operators/SharedQueue.cpp +++ b/dbms/src/Operators/SharedQueue.cpp @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include +#include #include #include From 751b852fb7cd3d6e2b43cda47646e4bbd35f7d24 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 29 May 2025 11:26:23 +0800 Subject: [PATCH 045/118] fix --- dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index b80a20226b1..603e2f0a6dd 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -48,7 +48,9 @@ PhysicalPlanNodePtr PhysicalCTESource::build( fine_grained_shuffle, log->identifier(), Block(schema), - cte_source.cte_id()); + cte_source.cte_id(), + cte_source.cte_sink_num(), + cte_source.cte_source_num()); } void PhysicalCTESource::buildPipelineExecGroupImpl( From e087d9e13bdd07e1f6e05fd17b65a27964e322c6 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 29 May 2025 16:39:24 +0800 Subject: [PATCH 046/118] fix some bugs --- dbms/src/Flash/Mpp/CTEManager.cpp | 8 +++++++- dbms/src/Flash/Mpp/CTEManager.h | 13 ++++++------- dbms/src/Flash/Mpp/MPPTask.cpp | 5 ----- dbms/src/Operators/CTE.cpp | 13 ++++++++++++- dbms/src/Operators/CTE.h | 14 ++++++++------ dbms/src/Operators/CTEReader.cpp | 28 +++++++++++++++++----------- dbms/src/Operators/CTEReader.h | 11 +++++++---- dbms/src/Operators/CTESinkOp.cpp | 1 - dbms/src/Operators/CTESinkOp.h | 1 - dbms/src/Operators/CTESource.cpp | 2 ++ 10 files changed, 59 insertions(+), 37 deletions(-) diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index eb922a6c447..daf7a33a5fb 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -32,7 +32,10 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id, const St if unlikely (iter_for_cte == iter->second.end()) throw Exception(fmt::format("Can't find cte: {}, partition: {}", query_id_and_cte_id, partition_id)); + auto * log = &Poco::Logger::get("LRUCache"); + iter_for_cte->second.sourceExit(); + LOG_INFO(log, "xzxdebug exit source num: {}, total exit: {}, total expect: {}", iter_for_cte->second.getSourceExitNum(), iter_for_cte->second.getTotalExitNum(), iter_for_cte->second.getExpectedTotalNum()); if (iter_for_cte->second.getTotalExitNum() == iter_for_cte->second.getExpectedTotalNum()) iter->second.erase(iter_for_cte); @@ -62,8 +65,9 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin while (iter_for_cte != iter_for_cte_end) { CTEWithCounter & cte_with_counter = iter_for_cte->second; - cte_with_counter.getCTE()->addRespAndNotifyEOF(resp); + cte_with_counter.getCTE()->addResp(resp); cte_with_counter.sinkExit(); + LOG_INFO(log, "xzxdebug exit sink num: {}, expect: {}, total exit: {}, total expect: {}", cte_with_counter.getSinkExitNum(), cte_with_counter.getExpectedSinkNum(), cte_with_counter.getTotalExitNum(), cte_with_counter.getExpectedTotalNum()); if (cte_with_counter.getSinkExitNum() == cte_with_counter.getExpectedSinkNum()) cte_with_counter.getCTE()->notifyEOF(); if (cte_with_counter.getTotalExitNum() == cte_with_counter.getExpectedTotalNum()) @@ -109,6 +113,8 @@ std::shared_ptr CTEManager::getCTEimpl( // It's the first time we request for the specific cte // Create it because no one created it before. auto cte = std::make_shared(); + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, "xzxdebug cte is ctreated expected_sink_num: {}, expected_source_num: {}, query_id_and_cte_id: {}", expected_sink_num, expected_source_num, query_id_and_cte_id); CTEWithCounter cte_with_counter(cte, expected_sink_num, expected_source_num); this->ctes[query_id_and_cte_id].insert(std::make_pair(partition_id, cte_with_counter)); return cte; diff --git a/dbms/src/Flash/Mpp/CTEManager.h b/dbms/src/Flash/Mpp/CTEManager.h index 5d4e544e373..7d9fc2ff5e2 100644 --- a/dbms/src/Flash/Mpp/CTEManager.h +++ b/dbms/src/Flash/Mpp/CTEManager.h @@ -16,7 +16,6 @@ #include -#include #include #include @@ -28,8 +27,8 @@ class CTEWithCounter public: explicit CTEWithCounter(std::shared_ptr cte_, Int32 expected_total_sink_num_, Int32 expected_total_source_num_) : cte(cte_) - , expected_total_sink_num(expected_total_sink_num_) - , expected_total_source_num(expected_total_source_num_) + , expected_sink_num(expected_total_sink_num_) + , expected_source_num(expected_total_source_num_) {} void sinkExit() { this->sink_exit_num++; } @@ -39,8 +38,8 @@ class CTEWithCounter Int32 getSourceExitNum() const { return this->source_exit_num; } Int32 getTotalExitNum() const { return this->getSinkExitNum() + this->getSourceExitNum(); } - Int32 getExpectedSinkNum() const { return this->expected_total_sink_num; } - Int32 getExpectedTotalNum() const { return this->getExpectedSinkNum() + this->expected_total_source_num; } + Int32 getExpectedSinkNum() const { return this->expected_sink_num; } + Int32 getExpectedTotalNum() const { return this->getExpectedSinkNum() + this->expected_source_num; } std::shared_ptr getCTE() const { return this->cte; } @@ -49,8 +48,8 @@ class CTEWithCounter Int32 sink_exit_num = 0; Int32 source_exit_num = 0; - Int32 expected_total_sink_num; - Int32 expected_total_source_num; + Int32 expected_sink_num; + Int32 expected_source_num; }; // TODO Test this class with UT diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 567c8e87a52..c743d8824ca 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -246,8 +246,6 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) if unlikely (!dag_context->dag_request.rootExecutor().has_cte_sink()) throw Exception("Task has either exchange sender or cte sink"); - LOG_INFO(log, "xzxdebug register cte sink"); - // There is no need to register tunnel for cte sink this->has_cte_sink = true; return; @@ -587,9 +585,7 @@ void MPPTask::runImpl() dag_context->tunnel_set->getExternalThreadCnt(), new_thread_count_of_mpp_receiver); - LOG_INFO(log, "xzxdebug wait for schedule"); scheduleOrWait(); - LOG_INFO(log, "xzxdebug start to run"); auto time_cost_in_schedule_ns = stopwatch.elapsed() - time_cost_in_preprocess_ns; dag_context->minTSO_wait_time_ns = time_cost_in_schedule_ns; @@ -621,7 +617,6 @@ void MPPTask::runImpl() #endif auto result = query_executor_holder->execute(); - LOG_INFO(log, "xzxdebug execution is finished"); auto log_level = Poco::Message::PRIO_DEBUG; if (!result.is_success || status != RUNNING) log_level = Poco::Message::PRIO_INFORMATION; diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 060783668bf..1003028360f 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -25,7 +25,7 @@ FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) { assert(queue.empty()); - std::shared_lock lock(this->rw_lock); + std::unique_lock lock(this->rw_lock); // TODO back to shared_lock if unlikely (this->is_cancelled) return FetchStatus::Cancelled; @@ -33,7 +33,15 @@ FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) if (block_num <= idx) { if (this->is_eof) + { + if (!this->first_print) + { + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, fmt::format("xzxdebug CTE returns eof, block num: {}, row num: {}", this->block_num, this->row_num)); + this->first_print = true; + } return FetchStatus::Eof; + } else return FetchStatus::Waiting; } @@ -49,6 +57,9 @@ bool CTE::pushBlock(const Block & block) if unlikely (this->is_cancelled) return false; + if unlikely (block.rows() == 0) + return true; + this->memory_usage += block.bytes(); if unlikely (this->blocks.empty()) this->pipe_cv.notifyAll(); diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 38f9214cc92..5ec02827558 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -39,7 +39,7 @@ class CTE : public NotifyFuture ~CTE() override { auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, fmt::format("xzxdebug block num: {}, row num: {}", this->block_num, this->row_num)); + LOG_INFO(log, fmt::format("xzxdebug CTE is destructured block num: {}, row num: {}", this->block_num, this->row_num)); } FetchStatus tryGetBunchBlocks(size_t idx, std::deque & queue); @@ -49,11 +49,10 @@ class CTE : public NotifyFuture void registerTask(TaskPtr && task) override; - void addRespAndNotifyEOF(const tipb::SelectResponse & resp) + void addResp(const tipb::SelectResponse & resp) { std::unique_lock lock(this->rw_lock); this->resp.MergeFrom(resp); - this->notifyEOFNoLock(); } void tryToGetResp(tipb::SelectResponse & resp) @@ -72,8 +71,6 @@ class CTE : public NotifyFuture } private: - void notifyEOFNoLock() { this->notifyImpl(true); } - template void notifyImpl(bool is_eof) { @@ -81,8 +78,13 @@ class CTE : public NotifyFuture if constexpr (has_lock) lock.lock(); + auto * log = &Poco::Logger::get("LRUCache"); + if likely (is_eof) + { + LOG_INFO(log, "xzxdebug set eof true"); this->is_eof = true; + } else this->is_cancelled = true; @@ -95,7 +97,7 @@ class CTE : public NotifyFuture std::shared_mutex rw_lock; Blocks blocks; - + bool first_print = false; // TODO remove size_t memory_usage = 0; // Tasks in WAITING_FOR_NOTIFY are saved in this deque diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 71ffb7ec74b..bd880dfec92 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -22,24 +22,27 @@ namespace DB std::pair CTEReader::fetchNextBlock() { std::lock_guard lock(this->mu); + auto * log = &Poco::Logger::get("LRUCache"); if (!this->blocks.empty()) { Block block = std::move(this->blocks.front()); this->blocks.pop_front(); this->output_block_num++; this->output_row_num += block.rows(); + if (!block) + LOG_INFO(log, "xzxdebug output empty block"); return {FetchStatus::Ok, block}; } auto ret = this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); - // if (ret != FetchStatus::Ok) - // { - // auto * log = &Poco::Logger::get("LRUCache"); - // LOG_INFO(log, "xzxdebug block_fetch_idx: {}, block num in cte: {}", this->block_fetch_idx, this->cte->blockNumForTest()); - // } switch (ret) { case FetchStatus::Eof: + if (!this->print_eof) + { + LOG_INFO(log, "xzxdebug block_fetch_idx: {}, block num in cte: {}", this->block_fetch_idx, this->cte->blockNumForTest()); + this->print_eof = true; + } if (this->resp.execution_summaries_size() == 0) this->cte->tryToGetResp(this->resp); case FetchStatus::Waiting: @@ -52,6 +55,8 @@ std::pair CTEReader::fetchNextBlock() this->blocks.pop_front(); this->output_block_num++; this->output_row_num += block.rows(); + if (!block) + LOG_INFO(log, "xzxdebug output empty block"); return {ret, block}; } throw Exception("Should not reach here"); @@ -63,17 +68,18 @@ FetchStatus CTEReader::checkAvailableBlock() if (!this->blocks.empty()) return FetchStatus::Ok; + auto * log = &Poco::Logger::get("LRUCache"); auto ret = this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); - // if (ret != FetchStatus::Ok) - // { - // auto * log = &Poco::Logger::get("LRUCache"); - // LOG_INFO(log, "xzxdebug block_fetch_idx: {}, block num in cte: {}", this->block_fetch_idx, this->cte->blockNumForTest()); - // } switch (ret) { + case FetchStatus::Eof: + if (!this->print_eof) + { + LOG_INFO(log, "xzxdebug block_fetch_idx: {}, block num in cte: {}", this->block_fetch_idx, this->cte->blockNumForTest()); + this->print_eof = true; + } case FetchStatus::Waiting: case FetchStatus::Cancelled: - case FetchStatus::Eof: return ret; case FetchStatus::Ok: this->save_block_num += this->blocks.size(); diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index b2d02c3c164..7e675eac1e3 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -54,12 +55,12 @@ class CTEReader LOG_INFO( log, fmt::format( - "xzxdebug query_id: {}, pid: {}, output block num: {}, output row num: {}, save block num:{}", - this->query_id_and_cte_id, - this->partition_id, + "xzxdebug output block num: {}, output row num: {}, save block num:{}, query_id: {}, pid: {}", this->output_block_num, this->output_row_num, - this->save_block_num)); + this->save_block_num, + this->query_id_and_cte_id, + this->partition_id)); } std::pair fetchNextBlock(); @@ -98,6 +99,8 @@ class CTEReader CTEManager * cte_manager; std::shared_ptr cte; + bool print_eof = false; + std::mutex mu; std::deque blocks; size_t block_fetch_idx = 0; diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 71b3e8642f5..12c28a456ba 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -26,7 +26,6 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) { if (!block) { - this->input_done = true; return OperatorStatus::FINISHED; } this->total_rows += block.rows(); diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 480b94a5437..10bb689e223 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -38,6 +38,5 @@ class CTESinkOp : public SinkOp private: std::shared_ptr cte; size_t total_rows = 0; - bool input_done = false; }; } // namespace DB diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index a819fdda473..e0b0df605d1 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -33,6 +33,8 @@ OperatorStatus CTESourceOp::readImpl(Block & block) if (this->resp.execution_summaries_size() != 0) this->io_profile_info->remote_execution_summary.add(this->resp); case FetchStatus::Ok: + if (!res.second) + LOG_INFO(log, "xzxdebug get empty block"); block = res.second; this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; From a4cd337881270049e289b00521f5b122f0b075c5 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 3 Jun 2025 16:38:41 +0800 Subject: [PATCH 047/118] fix cancel bug and remove xzxdebug --- contrib/tipb | 2 +- dbms/src/Flash/Mpp/CTEManager.cpp | 18 ++---------------- dbms/src/Flash/Mpp/MPPTask.cpp | 5 ----- dbms/src/Operators/CTE.cpp | 12 +----------- dbms/src/Operators/CTE.h | 21 --------------------- dbms/src/Operators/CTEReader.cpp | 22 ---------------------- dbms/src/Operators/CTEReader.h | 23 ----------------------- dbms/src/Operators/CTESource.cpp | 2 -- 8 files changed, 4 insertions(+), 101 deletions(-) diff --git a/contrib/tipb b/contrib/tipb index d2c561dad34..fbcfb7f65ab 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit d2c561dad347b35ef1d7ee2f57ecd9f3123bbdcf +Subproject commit fbcfb7f65ab6668ceb2d0e9ff1ec0d7d4ca9bf8b diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index daf7a33a5fb..9c19702b471 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -16,7 +16,6 @@ #include #include -#include #include #include @@ -27,24 +26,17 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id, const St std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if unlikely (iter == this->ctes.end()) - throw Exception(fmt::format("Can't find cte: {}", query_id_and_cte_id)); + // Maybe the task is cancelled and all ctes have been released + return; auto iter_for_cte = iter->second.find(partition_id); if unlikely (iter_for_cte == iter->second.end()) throw Exception(fmt::format("Can't find cte: {}, partition: {}", query_id_and_cte_id, partition_id)); - auto * log = &Poco::Logger::get("LRUCache"); - - iter_for_cte->second.sourceExit(); - LOG_INFO(log, "xzxdebug exit source num: {}, total exit: {}, total expect: {}", iter_for_cte->second.getSourceExitNum(), iter_for_cte->second.getTotalExitNum(), iter_for_cte->second.getExpectedTotalNum()); if (iter_for_cte->second.getTotalExitNum() == iter_for_cte->second.getExpectedTotalNum()) iter->second.erase(iter_for_cte); if (iter->second.size() == 0) - { - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, "xzxdebug releaseCTEBySource, erase {}", query_id_and_cte_id); this->ctes.erase(iter); - } } // TODO refine codes here, do not directly use map @@ -56,9 +48,6 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin // Maybe the task is cancelled and all ctes have been released return; - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, "xzxdebug releaseCTEBySink, counter"); - auto iter_for_cte = iter->second.begin(); auto iter_for_cte_end = iter->second.end(); std::vector ctes_need_erase; @@ -67,7 +56,6 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin CTEWithCounter & cte_with_counter = iter_for_cte->second; cte_with_counter.getCTE()->addResp(resp); cte_with_counter.sinkExit(); - LOG_INFO(log, "xzxdebug exit sink num: {}, expect: {}, total exit: {}, total expect: {}", cte_with_counter.getSinkExitNum(), cte_with_counter.getExpectedSinkNum(), cte_with_counter.getTotalExitNum(), cte_with_counter.getExpectedTotalNum()); if (cte_with_counter.getSinkExitNum() == cte_with_counter.getExpectedSinkNum()) cte_with_counter.getCTE()->notifyEOF(); if (cte_with_counter.getTotalExitNum() == cte_with_counter.getExpectedTotalNum()) @@ -113,8 +101,6 @@ std::shared_ptr CTEManager::getCTEimpl( // It's the first time we request for the specific cte // Create it because no one created it before. auto cte = std::make_shared(); - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, "xzxdebug cte is ctreated expected_sink_num: {}, expected_source_num: {}, query_id_and_cte_id: {}", expected_sink_num, expected_source_num, query_id_and_cte_id); CTEWithCounter cte_with_counter(cte, expected_sink_num, expected_source_num); this->ctes[query_id_and_cte_id].insert(std::make_pair(partition_id, cte_with_counter)); return cte; diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index c743d8824ca..d9d93c61e6c 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -209,11 +209,6 @@ void MPPTask::finishWrite() { if (this->has_cte_sink) { - LOG_INFO( - log, - "xzxdebug enter finishWrite, query_id_and_cte_id: {}, mpptask id: {}", - this->dag_context->getQueryIDAndCTEID(), - id.toString()); tipb::SelectResponse resp; if (dag_context->collect_execution_summaries) resp = mpp_task_statistics.genExecutionSummaryResponse(); diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 1003028360f..367e0880b9a 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -25,7 +25,7 @@ FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) { assert(queue.empty()); - std::unique_lock lock(this->rw_lock); // TODO back to shared_lock + std::shared_lock lock(this->rw_lock); if unlikely (this->is_cancelled) return FetchStatus::Cancelled; @@ -33,15 +33,7 @@ FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) if (block_num <= idx) { if (this->is_eof) - { - if (!this->first_print) - { - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, fmt::format("xzxdebug CTE returns eof, block num: {}, row num: {}", this->block_num, this->row_num)); - this->first_print = true; - } return FetchStatus::Eof; - } else return FetchStatus::Waiting; } @@ -64,8 +56,6 @@ bool CTE::pushBlock(const Block & block) if unlikely (this->blocks.empty()) this->pipe_cv.notifyAll(); this->blocks.push_back(block); - this->block_num++; - this->row_num += block.rows(); return true; } diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 5ec02827558..e540ce5bd91 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -36,12 +36,6 @@ enum class FetchStatus class CTE : public NotifyFuture { public: - ~CTE() override - { - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, fmt::format("xzxdebug CTE is destructured block num: {}, row num: {}", this->block_num, this->row_num)); - } - FetchStatus tryGetBunchBlocks(size_t idx, std::deque & queue); bool pushBlock(const Block & block); void notifyEOF() { this->notifyImpl(true); } @@ -64,12 +58,6 @@ class CTE : public NotifyFuture } } - Int64 blockNumForTest() - { - std::unique_lock lock(this->rw_lock); - return this->blocks.size(); - } - private: template void notifyImpl(bool is_eof) @@ -78,13 +66,8 @@ class CTE : public NotifyFuture if constexpr (has_lock) lock.lock(); - auto * log = &Poco::Logger::get("LRUCache"); - if likely (is_eof) - { - LOG_INFO(log, "xzxdebug set eof true"); this->is_eof = true; - } else this->is_cancelled = true; @@ -97,7 +80,6 @@ class CTE : public NotifyFuture std::shared_mutex rw_lock; Blocks blocks; - bool first_print = false; // TODO remove size_t memory_usage = 0; // Tasks in WAITING_FOR_NOTIFY are saved in this deque @@ -109,8 +91,5 @@ class CTE : public NotifyFuture bool get_resp = false; tipb::SelectResponse resp; - - Int64 block_num = 0; - Int64 row_num = 0; }; } // namespace DB diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index bd880dfec92..295ee141f84 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -22,15 +22,10 @@ namespace DB std::pair CTEReader::fetchNextBlock() { std::lock_guard lock(this->mu); - auto * log = &Poco::Logger::get("LRUCache"); if (!this->blocks.empty()) { Block block = std::move(this->blocks.front()); this->blocks.pop_front(); - this->output_block_num++; - this->output_row_num += block.rows(); - if (!block) - LOG_INFO(log, "xzxdebug output empty block"); return {FetchStatus::Ok, block}; } @@ -38,25 +33,15 @@ std::pair CTEReader::fetchNextBlock() switch (ret) { case FetchStatus::Eof: - if (!this->print_eof) - { - LOG_INFO(log, "xzxdebug block_fetch_idx: {}, block num in cte: {}", this->block_fetch_idx, this->cte->blockNumForTest()); - this->print_eof = true; - } if (this->resp.execution_summaries_size() == 0) this->cte->tryToGetResp(this->resp); case FetchStatus::Waiting: case FetchStatus::Cancelled: return {ret, Block()}; case FetchStatus::Ok: - this->save_block_num += this->blocks.size(); this->block_fetch_idx += this->blocks.size(); Block block = std::move(this->blocks.front()); this->blocks.pop_front(); - this->output_block_num++; - this->output_row_num += block.rows(); - if (!block) - LOG_INFO(log, "xzxdebug output empty block"); return {ret, block}; } throw Exception("Should not reach here"); @@ -68,21 +53,14 @@ FetchStatus CTEReader::checkAvailableBlock() if (!this->blocks.empty()) return FetchStatus::Ok; - auto * log = &Poco::Logger::get("LRUCache"); auto ret = this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); switch (ret) { case FetchStatus::Eof: - if (!this->print_eof) - { - LOG_INFO(log, "xzxdebug block_fetch_idx: {}, block num in cte: {}", this->block_fetch_idx, this->cte->blockNumForTest()); - this->print_eof = true; - } case FetchStatus::Waiting: case FetchStatus::Cancelled: return ret; case FetchStatus::Ok: - this->save_block_num += this->blocks.size(); this->block_fetch_idx += this->blocks.size(); return FetchStatus::Ok; } diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 7e675eac1e3..00af22c9ce6 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -19,15 +19,10 @@ #include #include -#include #include #include #include -// TODO remove them -#include -#include - namespace DB { class CTEReader @@ -50,17 +45,6 @@ class CTEReader { this->cte.reset(); this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id, this->partition_id); - - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO( - log, - fmt::format( - "xzxdebug output block num: {}, output row num: {}, save block num:{}, query_id: {}, pid: {}", - this->output_block_num, - this->output_row_num, - this->save_block_num, - this->query_id_and_cte_id, - this->partition_id)); } std::pair fetchNextBlock(); @@ -89,18 +73,11 @@ class CTEReader std::shared_ptr getCTE() const { return this->cte; } private: - Int64 output_block_num = 0; - Int64 output_row_num = 0; - - Int64 save_block_num = 0; - String query_id_and_cte_id; String partition_id; CTEManager * cte_manager; std::shared_ptr cte; - bool print_eof = false; - std::mutex mu; std::deque blocks; size_t block_fetch_idx = 0; diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index e0b0df605d1..a819fdda473 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -33,8 +33,6 @@ OperatorStatus CTESourceOp::readImpl(Block & block) if (this->resp.execution_summaries_size() != 0) this->io_profile_info->remote_execution_summary.add(this->resp); case FetchStatus::Ok: - if (!res.second) - LOG_INFO(log, "xzxdebug get empty block"); block = res.second; this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; From 8548f9752979f46dd8a406e511238674c9e150a1 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 10 Jun 2025 16:10:38 +0800 Subject: [PATCH 048/118] tweaking --- dbms/src/Operators/CTE.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 3e48acc3d60..1e1e6293a52 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -89,8 +89,15 @@ void CTE::spillBlocks() void CTE::registerTask(TaskPtr && task) { - // TODO can we directly register the task? Can we ensure that someone must wake it up? - pipe_cv.registerTask(std::move(task)); + { + std::unique_lock lock(this->rw_lock); + if (!this->hasDataNoLock()) + { + pipe_cv.registerTask(std::move(task)); + return; + } + } + this->pipe_cv.notifyTaskDirectly(std::move(task)); } CTE::CTEStatus CTE::getStatus() From b8f30dd8968905f3b2e30ced6a2c2055d50ea5db Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 10 Jun 2025 16:31:40 +0800 Subject: [PATCH 049/118] tweaking --- dbms/src/Operators/CTE.h | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 5b4445e2109..8ea75010598 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -97,7 +97,7 @@ class CTE : public NotifyFuture std::shared_mutex rw_lock; Blocks blocks; size_t memory_usage = 0; - size_t memory_threshold = 0; // TODO initialize it + size_t memory_threshold = 0; CTEStatus cte_status = CTEStatus::Normal; @@ -111,15 +111,12 @@ class CTE : public NotifyFuture PipeConditionVariable pipe_cv; CTESpill cte_spill; + bool is_spill_triggered = false; bool is_eof = false; -<<<<<<< HEAD - bool is_spill_triggered = false; -======= bool is_cancelled = false; bool get_resp = false; tipb::SelectResponse resp; ->>>>>>> cte }; } // namespace DB From e5e63785b3bd5ad9e2ca28b7cf293bf92bcd0a9f Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 10 Jun 2025 17:42:08 +0800 Subject: [PATCH 050/118] update tipb --- contrib/tipb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tipb b/contrib/tipb index fbcfb7f65ab..64577b0ef8d 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit fbcfb7f65ab6668ceb2d0e9ff1ec0d7d4ca9bf8b +Subproject commit 64577b0ef8da1bc85f1813108c0c1cd960f31f17 From 2f81b3a63602a7a78d5d8884e64ffd127e7099ae Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 11 Jun 2025 14:33:15 +0800 Subject: [PATCH 051/118] update tipb --- contrib/tipb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tipb b/contrib/tipb index 775c2379cbc..64577b0ef8d 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit 775c2379cbc717451d5b9c99ae26c753aeb4e8b0 +Subproject commit 64577b0ef8da1bc85f1813108c0c1cd960f31f17 From 03aa4e6d94d0339bd5fc3379f826e22d430c6adf Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 11 Jun 2025 14:55:26 +0800 Subject: [PATCH 052/118] modify get chunk interface --- dbms/src/Operators/CTE.cpp | 8 ++------ dbms/src/Operators/CTE.h | 2 +- dbms/src/Operators/CTEReader.cpp | 22 ++++++---------------- dbms/src/Operators/CTEReader.h | 2 -- 4 files changed, 9 insertions(+), 25 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 367e0880b9a..dd1b27dd59d 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -15,16 +15,13 @@ #include #include -#include #include #include namespace DB { -FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) +FetchStatus CTE::tryGetBlockAt(size_t idx, Block & block) { - assert(queue.empty()); - std::shared_lock lock(this->rw_lock); if unlikely (this->is_cancelled) return FetchStatus::Cancelled; @@ -38,8 +35,7 @@ FetchStatus CTE::tryGetBunchBlocks(size_t idx, std::deque & queue) return FetchStatus::Waiting; } - for (size_t i = idx; i < block_num; i++) - queue.push_back(this->blocks[i]); + block = this->blocks[idx]; return FetchStatus::Ok; } diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index e540ce5bd91..afbf273f50a 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -36,7 +36,7 @@ enum class FetchStatus class CTE : public NotifyFuture { public: - FetchStatus tryGetBunchBlocks(size_t idx, std::deque & queue); + FetchStatus tryGetBlockAt(size_t idx, Block & block); bool pushBlock(const Block & block); void notifyEOF() { this->notifyImpl(true); } void notifyCancel() { this->notifyImpl(false); } diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 295ee141f84..f81cd236c64 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -21,15 +21,9 @@ namespace DB { std::pair CTEReader::fetchNextBlock() { + Block block; std::lock_guard lock(this->mu); - if (!this->blocks.empty()) - { - Block block = std::move(this->blocks.front()); - this->blocks.pop_front(); - return {FetchStatus::Ok, block}; - } - - auto ret = this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); + auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx, block); switch (ret) { case FetchStatus::Eof: @@ -39,9 +33,7 @@ std::pair CTEReader::fetchNextBlock() case FetchStatus::Cancelled: return {ret, Block()}; case FetchStatus::Ok: - this->block_fetch_idx += this->blocks.size(); - Block block = std::move(this->blocks.front()); - this->blocks.pop_front(); + this->block_fetch_idx++; return {ret, block}; } throw Exception("Should not reach here"); @@ -49,11 +41,9 @@ std::pair CTEReader::fetchNextBlock() FetchStatus CTEReader::checkAvailableBlock() { + Block block; std::lock_guard lock(this->mu); - if (!this->blocks.empty()) - return FetchStatus::Ok; - - auto ret = this->cte->tryGetBunchBlocks(this->block_fetch_idx, this->blocks); + auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx, block); switch (ret) { case FetchStatus::Eof: @@ -61,7 +51,7 @@ FetchStatus CTEReader::checkAvailableBlock() case FetchStatus::Cancelled: return ret; case FetchStatus::Ok: - this->block_fetch_idx += this->blocks.size(); + // Do not add block_fetch_idx here as we only check if there are available blocks return FetchStatus::Ok; } throw Exception("Should not reach here"); diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 00af22c9ce6..e06d8ca8745 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -19,7 +19,6 @@ #include #include -#include #include #include @@ -79,7 +78,6 @@ class CTEReader std::shared_ptr cte; std::mutex mu; - std::deque blocks; size_t block_fetch_idx = 0; bool resp_fetched = false; From 6e8b133ee5db0e725525a0110117dd6230f42b56 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 11 Jun 2025 14:57:40 +0800 Subject: [PATCH 053/118] tweaking --- dbms/src/Operators/CTE.cpp | 10 +++++----- dbms/src/Operators/CTE.h | 4 ++-- dbms/src/Operators/CTEReader.cpp | 22 +++++++++++----------- dbms/src/Operators/CTEReader.h | 4 ++-- dbms/src/Operators/CTESource.cpp | 16 ++++++++-------- 5 files changed, 28 insertions(+), 28 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index dd1b27dd59d..97663a6971a 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -20,23 +20,23 @@ namespace DB { -FetchStatus CTE::tryGetBlockAt(size_t idx, Block & block) +Status CTE::tryGetBlockAt(size_t idx, Block & block) { std::shared_lock lock(this->rw_lock); if unlikely (this->is_cancelled) - return FetchStatus::Cancelled; + return Status::Cancelled; auto block_num = this->blocks.size(); if (block_num <= idx) { if (this->is_eof) - return FetchStatus::Eof; + return Status::Eof; else - return FetchStatus::Waiting; + return Status::Waiting; } block = this->blocks[idx]; - return FetchStatus::Ok; + return Status::Ok; } bool CTE::pushBlock(const Block & block) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index afbf273f50a..2f73068c26c 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -25,7 +25,7 @@ namespace DB { -enum class FetchStatus +enum class Status { Ok, Waiting, @@ -36,7 +36,7 @@ enum class FetchStatus class CTE : public NotifyFuture { public: - FetchStatus tryGetBlockAt(size_t idx, Block & block); + Status tryGetBlockAt(size_t idx, Block & block); bool pushBlock(const Block & block); void notifyEOF() { this->notifyImpl(true); } void notifyCancel() { this->notifyImpl(false); } diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index f81cd236c64..319e899fb49 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -19,40 +19,40 @@ namespace DB { -std::pair CTEReader::fetchNextBlock() +std::pair CTEReader::fetchNextBlock() { Block block; std::lock_guard lock(this->mu); auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx, block); switch (ret) { - case FetchStatus::Eof: + case Status::Eof: if (this->resp.execution_summaries_size() == 0) this->cte->tryToGetResp(this->resp); - case FetchStatus::Waiting: - case FetchStatus::Cancelled: + case Status::Waiting: + case Status::Cancelled: return {ret, Block()}; - case FetchStatus::Ok: + case Status::Ok: this->block_fetch_idx++; return {ret, block}; } throw Exception("Should not reach here"); } -FetchStatus CTEReader::checkAvailableBlock() +Status CTEReader::checkAvailableBlock() { Block block; std::lock_guard lock(this->mu); auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx, block); switch (ret) { - case FetchStatus::Eof: - case FetchStatus::Waiting: - case FetchStatus::Cancelled: + case Status::Eof: + case Status::Waiting: + case Status::Cancelled: return ret; - case FetchStatus::Ok: + case Status::Ok: // Do not add block_fetch_idx here as we only check if there are available blocks - return FetchStatus::Ok; + return Status::Ok; } throw Exception("Should not reach here"); } diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index e06d8ca8745..8903491b8c4 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -46,8 +46,8 @@ class CTEReader this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id, this->partition_id); } - std::pair fetchNextBlock(); - FetchStatus checkAvailableBlock(); + std::pair fetchNextBlock(); + Status checkAvailableBlock(); void getResp(tipb::SelectResponse & resp) { diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index a819fdda473..07184274011 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -28,15 +28,15 @@ OperatorStatus CTESourceOp::readImpl(Block & block) auto res = this->cte_reader->fetchNextBlock(); switch (res.first) { - case FetchStatus::Eof: + case Status::Eof: this->cte_reader->getResp(this->resp); if (this->resp.execution_summaries_size() != 0) this->io_profile_info->remote_execution_summary.add(this->resp); - case FetchStatus::Ok: + case Status::Ok: block = res.second; this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; - case FetchStatus::Waiting: + case Status::Waiting: if likely (this->cte_reader->isBlockGenerated()) { return OperatorStatus::WAITING; @@ -48,7 +48,7 @@ OperatorStatus CTESourceOp::readImpl(Block & block) this->cte_reader->setNotifyFuture(); return OperatorStatus::WAIT_FOR_NOTIFY; } - case FetchStatus::Cancelled: + case Status::Cancelled: return OperatorStatus::CANCELLED; } } @@ -58,12 +58,12 @@ OperatorStatus CTESourceOp::awaitImpl() auto res = this->cte_reader->checkAvailableBlock(); switch (res) { - case DB::FetchStatus::Eof: - case DB::FetchStatus::Ok: + case DB::Status::Eof: + case DB::Status::Ok: return OperatorStatus::HAS_OUTPUT; - case DB::FetchStatus::Waiting: + case DB::Status::Waiting: return OperatorStatus::WAITING; - case DB::FetchStatus::Cancelled: + case DB::Status::Cancelled: return OperatorStatus::CANCELLED; } } From 676f7436211ad491f63680f948f9e539f9b7ec64 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 11 Jun 2025 20:53:28 +0800 Subject: [PATCH 054/118] refactor codes --- dbms/src/Core/CTESpill.cpp | 24 ++++--- dbms/src/Core/CTESpill.h | 4 +- dbms/src/Operators/CTE.cpp | 120 +++++++++++++++++++++++++++---- dbms/src/Operators/CTE.h | 9 ++- dbms/src/Operators/CTEReader.cpp | 37 +++++----- dbms/src/Operators/CTEReader.h | 7 +- dbms/src/Operators/CTESinkOp.cpp | 4 +- dbms/src/Operators/CTESinkOp.h | 2 +- dbms/src/Operators/CTESource.cpp | 44 ++++++++---- dbms/src/Operators/CTESource.h | 5 +- 10 files changed, 184 insertions(+), 72 deletions(-) diff --git a/dbms/src/Core/CTESpill.cpp b/dbms/src/Core/CTESpill.cpp index 4903af267d2..9b440e650c4 100644 --- a/dbms/src/Core/CTESpill.cpp +++ b/dbms/src/Core/CTESpill.cpp @@ -40,7 +40,7 @@ void CTESpill::writeBlocks(const Blocks & blocks) prev_block_offset = this->block_offsets.back().second; const auto block_size = this->write_streams.back().writeAndReturnBlockSize(block); - this->block_offsets.push_back((std::make_pair(this->spilled_files.size()-1, prev_block_offset+block_size))); + this->block_offsets.push_back((std::make_pair(this->spilled_files.size() - 1, prev_block_offset + block_size))); } } @@ -48,8 +48,11 @@ Block CTESpill::readBlockAt(Int64 idx) { std::shared_lock lock(this->rw_lock); if unlikely (idx >= static_cast(this->block_offsets.size())) - throw Exception(fmt::format("Requested block idx({}) is larger than total block number({})", idx, this->block_offsets.size())); - + throw Exception(fmt::format( + "Requested block idx({}) is larger than total block number({})", + idx, + this->block_offsets.size())); + auto block_location = this->block_offsets[idx]; Int64 stream_idx = block_location.first; Int64 block_offset = block_location.second; @@ -71,15 +74,18 @@ Int64 CTESpill::blockNum() Int64 CTESpill::getBlockSizeNoLock(Int64 idx) const { if unlikely (idx >= static_cast(this->block_offsets.size())) - throw Exception(fmt::format("Requested block idx({}) is larger than total block number({})", idx, this->block_offsets.size())); - + throw Exception(fmt::format( + "Requested block idx({}) is larger than total block number({})", + idx, + this->block_offsets.size())); + if unlikely (idx == 0) return this->block_offsets[0].second; - - Int64 prev_block_file_idx = this->block_offsets[idx-1].first; + + Int64 prev_block_file_idx = this->block_offsets[idx - 1].first; if unlikely (prev_block_file_idx != this->block_offsets[idx].first) return this->block_offsets[idx].second; - - return this->block_offsets[idx].second - this->block_offsets[idx-1].second; + + return this->block_offsets[idx].second - this->block_offsets[idx - 1].second; } } // namespace DB diff --git a/dbms/src/Core/CTESpill.h b/dbms/src/Core/CTESpill.h index 36bb8040a0b..5e0976495cd 100644 --- a/dbms/src/Core/CTESpill.h +++ b/dbms/src/Core/CTESpill.h @@ -16,10 +16,10 @@ #include #include -#include -#include #include #include +#include +#include #include #include diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index e58d8089c72..cd8fd9e9f30 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include #include +#include #include #include @@ -22,50 +22,137 @@ namespace DB { -FetchStatus CTE::tryGetBlockAt(size_t idx, Block & block) +Status CTE::tryGetBlockAt(size_t idx, Block & block) { + { + std::shared_lock status_lock(this->aux_rw_lock); + if (this->cte_status != CTE::Normal) + return Status::IOOut; + } + std::shared_lock lock(this->rw_lock); + if unlikely (this->is_cancelled) return Status::Cancelled; + if (this->is_spill_triggered) + { + auto spilled_block_num = static_cast(this->cte_spill.blockNum()); + if (idx < spilled_block_num) + return Status::IOIn; + + idx -= spilled_block_num; + } + auto block_num = this->blocks.size(); if (block_num <= idx) { if (this->is_eof) - return {Status::Eof, Block()}; + return Status::Eof; else - return {Status::BlockUnavailable, Block()}; + return Status::BlockUnavailable; } block = this->blocks[idx]; - return FetchStatus::Ok; + return Status::Ok; } -bool CTE::pushBlock(const Block & block) +Status CTE::checkAvailableBlock(size_t idx) { - std::unique_lock lock(this->rw_lock); + { + std::shared_lock status_lock(this->aux_rw_lock); + if (this->cte_status != CTE::Normal) + return Status::BlockUnavailable; + } + + std::shared_lock lock(this->rw_lock); + if unlikely (this->is_cancelled) - return false; + return Status::Cancelled; + + if (this->is_spill_triggered) + { + auto spilled_block_num = static_cast(this->cte_spill.blockNum()); + if (idx < spilled_block_num) + return Status::Ok; + + idx -= spilled_block_num; + } + + if (this->blocks.size() > idx) + return Status::Ok; + return Status::BlockUnavailable; +} + +Status CTE::pushBlock(const Block & block) +{ + std::unique_lock lock(this->rw_lock, std::defer_lock); + Status ret = Status::Ok; + { + std::unique_lock status_lock(this->aux_rw_lock); + if (this->cte_status != CTE::Normal) + { + // Block memory usage will be calculated after the finish of spill + this->tmp_blocks.push_back(block); + return Status::IOOut; + } + + // This function is called in cpu pool, we don't want to wait for this lock too long. + // This lock may be held when spill is in execution. So we need ensure that cte status is not changed + lock.lock(); - if unlikely (block.rows() == 0) - return true; + if unlikely (this->is_cancelled) + return Status::Cancelled; - if unlikely (this->blocks.empty()) + if unlikely (block.rows() == 0) + // All rows in block may have been filtered and it's needles to store this block + return Status::Ok; + + this->memory_usage += block.bytes(); + if (this->memory_usage >= this->memory_threshold) + { + this->cte_status = CTE::NeedSpill; + ret = Status::IOOut; + } + } + + if unlikely (!this->hasDataNoLock()) + // It's the first time to get block; wake up all tasks that are waiting for blocks. this->pipe_cv.notifyAll(); this->blocks.push_back(block); - return true; + return ret; +} + +Status CTE::getBlockFromDisk(size_t idx, Block & block) +{ + std::shared_lock lock(this->rw_lock); + if unlikely (this->is_cancelled) + return Status::Cancelled; + + if unlikely (!this->is_spill_triggered) + // We can call this function only when spill is triggered + throw Exception("Spill should be triggered"); + + if unlikely (static_cast(this->cte_spill.blockNum()) <= idx) + throw Exception("Requested block is not in disk"); + + block = this->cte_spill.readBlockAt(idx); + return Status::Ok; } -void CTE::spillBlocks() +bool CTE::spillBlocks() { std::unique_lock lock(this->rw_lock); + if unlikely (this->is_cancelled) + return false; + while (true) { this->cte_spill.writeBlocks(this->blocks); this->blocks.clear(); this->memory_usage = 0; - + std::unique_lock aux_lock(this->aux_rw_lock); for (const auto & block : this->tmp_blocks) { @@ -76,15 +163,20 @@ void CTE::spillBlocks() this->tmp_blocks.clear(); if (this->memory_usage < this->memory_threshold) + { + this->cte_status = CTEStatus::Normal; break; + } } // Many tasks may be waiting for the finish of spill this->pipe_cv.notifyAll(); + return true; } void CTE::registerTask(TaskPtr && task) { + // TODO sometimes we register task because of spill, consider this situation { std::unique_lock lock(this->rw_lock); if (!this->hasDataNoLock()) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index f235df7ed5c..957f59503fa 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -50,12 +50,14 @@ class CTE : public NotifyFuture CTEStatus getStatus(); Status tryGetBlockAt(size_t idx, Block & block); - bool pushBlock(const Block & block); + Status checkAvailableBlock(size_t idx); + Status pushBlock(const Block & block); + Status getBlockFromDisk(size_t idx, Block & block); + void notifyEOF() { this->notifyImpl(true); } void notifyCancel() { this->notifyImpl(false); } - // TODO should have return value to indicate if spill successes - void spillBlocks(); + bool spillBlocks(); void registerTask(TaskPtr && task) override; @@ -92,6 +94,7 @@ class CTE : public NotifyFuture } // Return true if CTE has data + // TODO should also consider blocks in the disk inline bool hasDataNoLock() const { return !this->blocks.empty(); } std::shared_mutex rw_lock; diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index f81cd236c64..f34e4aa1fbb 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -19,41 +19,40 @@ namespace DB { -std::pair CTEReader::fetchNextBlock() +std::pair CTEReader::fetchNextBlock() { Block block; std::lock_guard lock(this->mu); auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx, block); switch (ret) { - case FetchStatus::Eof: + case Status::Eof: if (this->resp.execution_summaries_size() == 0) this->cte->tryToGetResp(this->resp); - case FetchStatus::Waiting: - case FetchStatus::Cancelled: + case Status::IOOut: + case Status::IOIn: + case Status::Cancelled: + case Status::BlockUnavailable: return {ret, Block()}; - case FetchStatus::Ok: + case Status::Ok: this->block_fetch_idx++; return {ret, block}; } throw Exception("Should not reach here"); } -FetchStatus CTEReader::checkAvailableBlock() +Status CTEReader::fetchBlockFromDisk(Block & block) { - Block block; std::lock_guard lock(this->mu); - auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx, block); - switch (ret) - { - case FetchStatus::Eof: - case FetchStatus::Waiting: - case FetchStatus::Cancelled: - return ret; - case FetchStatus::Ok: - // Do not add block_fetch_idx here as we only check if there are available blocks - return FetchStatus::Ok; - } - throw Exception("Should not reach here"); + auto status = this->cte->getBlockFromDisk(this->block_fetch_idx, block); + if likely (status == Status::Ok) + this->block_fetch_idx++; + return status; +} + +Status CTEReader::checkAvailableBlock() +{ + std::lock_guard lock(this->mu); + return this->cte->checkAvailableBlock(this->block_fetch_idx); } } // namespace DB diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index e06d8ca8745..3e05859cc0a 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -46,8 +46,11 @@ class CTEReader this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id, this->partition_id); } - std::pair fetchNextBlock(); - FetchStatus checkAvailableBlock(); + std::pair fetchNextBlock(); + Status fetchBlockFromDisk(Block & block); + Status checkAvailableBlock(); + + CTE::CTEStatus getCTEStatus() const { return this->cte->getStatus(); } void getResp(tipb::SelectResponse & resp) { diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 4d565470903..a86251f586e 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -44,9 +44,7 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) OperatorStatus CTESinkOp::executeIOImpl() { - this->cte->spillBlocks(); - return OperatorStatus::NEED_INPUT; - if (this->cte->pushBlock(block)) + if likely (this->cte->spillBlocks()) return OperatorStatus::NEED_INPUT; return OperatorStatus::CANCELLED; } diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 6860e90ca98..8fc3a0a779d 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -14,10 +14,10 @@ #pragma once +#include #include #include #include -#include namespace DB { diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index 493bfac4cc0..56e4a77e801 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -25,6 +26,13 @@ void CTESourceOp::operateSuffixImpl() OperatorStatus CTESourceOp::readImpl(Block & block) { + if (this->block_from_disk) + { + block = this->block_from_disk; + this->block_from_disk.clear(); + return OperatorStatus::HAS_OUTPUT; + } + auto res = this->cte_reader->fetchNextBlock(); switch (res.first) { @@ -40,11 +48,11 @@ OperatorStatus CTESourceOp::readImpl(Block & block) // Expected block is in disk, we need to read it from disk return OperatorStatus::IO_IN; case Status::IOOut: - { - // CTE is spilling blocks to disk, we need to wait the finish of spill - this->wait_type = CTESourceOp::Spill; - return OperatorStatus::WAITING; - } + { + // CTE is spilling blocks to disk, we need to wait the finish of spill + this->wait_type = CTESourceOp::Spill; + return OperatorStatus::WAITING; + } case Status::BlockUnavailable: if likely (this->cte_reader->isBlockGenerated()) { @@ -64,9 +72,17 @@ OperatorStatus CTESourceOp::readImpl(Block & block) OperatorStatus CTESourceOp::executeIOImpl() { - this->block_from_disk = this->cte->getBlockFromDisk(this->block_fetch_idx); - this->block_fetch_idx++; - return OperatorStatus::HAS_OUTPUT; + RUNTIME_CHECK(!this->block_from_disk); + auto status = this->cte_reader->fetchBlockFromDisk(this->block_from_disk); + switch (status) + { + case Status::Ok: + return OperatorStatus::HAS_OUTPUT; + case Status::Cancelled: + return OperatorStatus::CANCELLED; + default: + throw Exception(fmt::format("Get unexpected status {}", magic_enum::enum_name(status))); + } } OperatorStatus CTESourceOp::awaitImpl() @@ -74,10 +90,8 @@ OperatorStatus CTESourceOp::awaitImpl() if (this->wait_type == CTESourceOp::WaitType::NeedMoreBlock) { auto res = this->cte_reader->checkAvailableBlock(); - switch (ret.first) + switch (res) { - case Status::IOOut: - this->wait_type = CTESourceOp::WaitType::Spill; case Status::BlockUnavailable: return OperatorStatus::WAITING; case Status::Ok: @@ -85,13 +99,13 @@ OperatorStatus CTESourceOp::awaitImpl() return OperatorStatus::HAS_OUTPUT; case Status::Cancelled: return OperatorStatus::CANCELLED; - case Status::IOIn: - return OperatorStatus::IO_IN; + default: + throw Exception(fmt::format("Get unexpected status {}", magic_enum::enum_name(res))); } } else if (this->wait_type == CTESourceOp::WaitType::Spill) { - switch (this->cte->getStatus()) + switch (this->cte_reader->getCTEStatus()) { case CTE::CTEStatus::Normal: return OperatorStatus::HAS_OUTPUT; @@ -102,7 +116,7 @@ OperatorStatus CTESourceOp::awaitImpl() } else { - throw Exception("Unexpected wait type"); + throw Exception(fmt::format("Unexpected wait type {}", magic_enum::enum_name(this->wait_type))); } } } // namespace DB diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 05b9989139b..48f4d354621 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -35,9 +35,9 @@ class CTESourceOp : public SourceOp std::shared_ptr cte_reader_, const NamesAndTypes & schema) : SourceOp(exec_context_, req_id) + , wait_type(NeedMoreBlock) , cte_reader(cte_reader_) , io_profile_info(IOProfileInfo::createForRemote(profile_info_ptr, 1)) - , wait_type(NeedMoreBlock) { setHeader(Block(getColumnWithTypeAndName(schema))); } @@ -61,9 +61,6 @@ class CTESourceOp : public SourceOp }; String query_id_and_cte_id; - CTEManager * cte_manager; - std::shared_ptr cte; - Block block_from_disk; uint64_t total_rows{}; From 1a23b99ef0dac7de2ffb3a6c11c34f3c02cd9d86 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 12 Jun 2025 16:12:39 +0800 Subject: [PATCH 055/118] refine --- dbms/src/Operators/CTE.cpp | 10 +++++----- dbms/src/Operators/CTE.h | 4 ++-- dbms/src/Operators/CTEReader.cpp | 22 +++++++++++----------- dbms/src/Operators/CTEReader.h | 4 ++-- dbms/src/Operators/CTESource.cpp | 16 ++++++++-------- dbms/src/Operators/CTESource.h | 2 +- 6 files changed, 29 insertions(+), 29 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 97663a6971a..eddfa0cf458 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -20,23 +20,23 @@ namespace DB { -Status CTE::tryGetBlockAt(size_t idx, Block & block) +CTEOpStatus CTE::tryGetBlockAt(size_t idx, Block & block) { std::shared_lock lock(this->rw_lock); if unlikely (this->is_cancelled) - return Status::Cancelled; + return CTEOpStatus::Cancelled; auto block_num = this->blocks.size(); if (block_num <= idx) { if (this->is_eof) - return Status::Eof; + return CTEOpStatus::Eof; else - return Status::Waiting; + return CTEOpStatus::Waiting; } block = this->blocks[idx]; - return Status::Ok; + return CTEOpStatus::Ok; } bool CTE::pushBlock(const Block & block) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 2f73068c26c..ebaecf5a1f4 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -25,7 +25,7 @@ namespace DB { -enum class Status +enum class CTEOpStatus { Ok, Waiting, @@ -36,7 +36,7 @@ enum class Status class CTE : public NotifyFuture { public: - Status tryGetBlockAt(size_t idx, Block & block); + CTEOpStatus tryGetBlockAt(size_t idx, Block & block); bool pushBlock(const Block & block); void notifyEOF() { this->notifyImpl(true); } void notifyCancel() { this->notifyImpl(false); } diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 319e899fb49..510074ec132 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -19,40 +19,40 @@ namespace DB { -std::pair CTEReader::fetchNextBlock() +std::pair CTEReader::fetchNextBlock() { Block block; std::lock_guard lock(this->mu); auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx, block); switch (ret) { - case Status::Eof: + case CTEOpStatus::Eof: if (this->resp.execution_summaries_size() == 0) this->cte->tryToGetResp(this->resp); - case Status::Waiting: - case Status::Cancelled: + case CTEOpStatus::Waiting: + case CTEOpStatus::Cancelled: return {ret, Block()}; - case Status::Ok: + case CTEOpStatus::Ok: this->block_fetch_idx++; return {ret, block}; } throw Exception("Should not reach here"); } -Status CTEReader::checkAvailableBlock() +CTEOpStatus CTEReader::checkAvailableBlock() { Block block; std::lock_guard lock(this->mu); auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx, block); switch (ret) { - case Status::Eof: - case Status::Waiting: - case Status::Cancelled: + case CTEOpStatus::Eof: + case CTEOpStatus::Waiting: + case CTEOpStatus::Cancelled: return ret; - case Status::Ok: + case CTEOpStatus::Ok: // Do not add block_fetch_idx here as we only check if there are available blocks - return Status::Ok; + return CTEOpStatus::Ok; } throw Exception("Should not reach here"); } diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 8903491b8c4..d204c5a2b5f 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -46,8 +46,8 @@ class CTEReader this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id, this->partition_id); } - std::pair fetchNextBlock(); - Status checkAvailableBlock(); + std::pair fetchNextBlock(); + CTEOpStatus checkAvailableBlock(); void getResp(tipb::SelectResponse & resp) { diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESource.cpp index 07184274011..d7cf8e44cab 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESource.cpp @@ -28,15 +28,15 @@ OperatorStatus CTESourceOp::readImpl(Block & block) auto res = this->cte_reader->fetchNextBlock(); switch (res.first) { - case Status::Eof: + case CTEOpStatus::Eof: this->cte_reader->getResp(this->resp); if (this->resp.execution_summaries_size() != 0) this->io_profile_info->remote_execution_summary.add(this->resp); - case Status::Ok: + case CTEOpStatus::Ok: block = res.second; this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; - case Status::Waiting: + case CTEOpStatus::Waiting: if likely (this->cte_reader->isBlockGenerated()) { return OperatorStatus::WAITING; @@ -48,7 +48,7 @@ OperatorStatus CTESourceOp::readImpl(Block & block) this->cte_reader->setNotifyFuture(); return OperatorStatus::WAIT_FOR_NOTIFY; } - case Status::Cancelled: + case CTEOpStatus::Cancelled: return OperatorStatus::CANCELLED; } } @@ -58,12 +58,12 @@ OperatorStatus CTESourceOp::awaitImpl() auto res = this->cte_reader->checkAvailableBlock(); switch (res) { - case DB::Status::Eof: - case DB::Status::Ok: + case CTEOpStatus::Eof: + case CTEOpStatus::Ok: return OperatorStatus::HAS_OUTPUT; - case DB::Status::Waiting: + case CTEOpStatus::Waiting: return OperatorStatus::WAITING; - case DB::Status::Cancelled: + case CTEOpStatus::Cancelled: return OperatorStatus::CANCELLED; } } diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 1762e506024..8661219c5a9 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -36,7 +36,7 @@ class CTESourceOp : public SourceOp const NamesAndTypes & schema) : SourceOp(exec_context_, req_id) , cte_reader(cte_reader_) - , io_profile_info(IOProfileInfo::createForRemote(profile_info_ptr, 1)) + , io_profile_info(IOProfileInfo::createForLocal(profile_info_ptr)) { setHeader(Block(getColumnWithTypeAndName(schema))); } From 2a1e45cb59974bb3b5f1851e4b72ba8e29d7731b Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 13 Jun 2025 15:39:38 +0800 Subject: [PATCH 056/118] tweaking --- dbms/src/Core/CTESpill.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Core/CTESpill.cpp b/dbms/src/Core/CTESpill.cpp index 9b440e650c4..686a3030b0c 100644 --- a/dbms/src/Core/CTESpill.cpp +++ b/dbms/src/Core/CTESpill.cpp @@ -58,6 +58,7 @@ Block CTESpill::readBlockAt(Int64 idx) Int64 block_offset = block_location.second; Int64 block_size = this->getBlockSizeNoLock(idx); + // TODO why we resize buf? Maybe we can delete this code. if (this->buf.size() < static_cast(block_size)) this->buf.resize(block_size); From f3d8d5fbbf49372e06d6ca0a1a8c8786b0397ccf Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 13 Jun 2025 15:40:46 +0800 Subject: [PATCH 057/118] tweaking --- dbms/src/Operators/CTESource.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESource.h index 8661219c5a9..1762e506024 100644 --- a/dbms/src/Operators/CTESource.h +++ b/dbms/src/Operators/CTESource.h @@ -36,7 +36,7 @@ class CTESourceOp : public SourceOp const NamesAndTypes & schema) : SourceOp(exec_context_, req_id) , cte_reader(cte_reader_) - , io_profile_info(IOProfileInfo::createForLocal(profile_info_ptr)) + , io_profile_info(IOProfileInfo::createForRemote(profile_info_ptr, 1)) { setHeader(Block(getColumnWithTypeAndName(schema))); } From 5dacc26038deaa48947301029c5e695b5a62eef4 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 16 Jun 2025 11:04:07 +0800 Subject: [PATCH 058/118] address comments --- dbms/src/Common/TiFlashMetrics.h | 3 +- dbms/src/Flash/Coprocessor/DAGContext.h | 4 +-- dbms/src/Flash/Executor/PipelineExecutor.cpp | 2 +- .../Flash/Executor/PipelineExecutorContext.h | 1 - dbms/src/Flash/Mpp/CTEManager.cpp | 29 +++++++++---------- dbms/src/Flash/Mpp/CTEManager.h | 8 ++--- dbms/src/Flash/Mpp/MPPTaskId.h | 2 +- .../Pipeline/Schedule/Reactor/WaitReactor.cpp | 1 + .../Schedule/Tasks/PipeConditionVariable.h | 3 ++ dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 1 + .../Flash/Planner/Plans/PhysicalCTESink.cpp | 1 - .../Flash/Planner/Plans/PhysicalCTESource.cpp | 3 +- dbms/src/Operators/CTE.cpp | 10 +++---- .../{CTESource.cpp => CTESourceOp.cpp} | 2 +- .../Operators/{CTESource.h => CTESourceOp.h} | 0 dbms/src/Operators/Operator.h | 1 - dbms/src/Operators/SharedQueue.cpp | 1 - 17 files changed, 35 insertions(+), 37 deletions(-) rename dbms/src/Operators/{CTESource.cpp => CTESourceOp.cpp} (98%) rename dbms/src/Operators/{CTESource.h => CTESourceOp.h} (100%) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 4895b9ab484..b3a6a3488f0 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -770,7 +770,8 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_wait_on_tunnel_sender_write, {"type", "wait_on_tunnel_sender_write"}), \ F(type_wait_on_join_build, {"type", "wait_on_join_build"}), \ F(type_wait_on_join_probe, {"type", "wait_on_join_probe"}), \ - F(type_wait_on_result_queue_write, {"type", "wait_on_result_queue_write"})) \ + F(type_wait_on_result_queue_write, {"type", "wait_on_result_queue_write"}), \ + F(type_wait_on_cte, {"type", "wait_on_cte"})) \ M(tiflash_pipeline_task_duration_seconds, \ "Bucketed histogram of pipeline task duration in seconds", \ Histogram, /* these command usually cost several hundred milliseconds to several seconds, increase the start bucket to 5ms */ \ diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 7f6f98447ad..37de63de639 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -365,11 +365,10 @@ class DAGContext void setQueryIDAndCTEID(const String & query_id_and_cte_id) { // MPP Task has only one CTESink, it's impossible to set query_id_and_cte_id twice - RUNTIME_CHECK(this->query_id_and_cte_id.empty()); + RUNTIME_CHECK(this->query_id_and_cte_id.empty(), this->query_id_and_cte_id); this->query_id_and_cte_id = query_id_and_cte_id; } - void sinkNeedRelease() { this->sink_need_release = true; } std::vector> getCTEs() const { return this->ctes; } void addCTE(std::shared_ptr & cte) { this->ctes.push_back(cte); } bool hasCTESource() const { return this->has_cte_source; } @@ -494,7 +493,6 @@ class DAGContext String query_id_and_cte_id; bool has_cte_source = false; - bool sink_need_release = false; std::vector> ctes; }; diff --git a/dbms/src/Flash/Executor/PipelineExecutor.cpp b/dbms/src/Flash/Executor/PipelineExecutor.cpp index 1f8104234ac..c41968e35ba 100644 --- a/dbms/src/Flash/Executor/PipelineExecutor.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutor.cpp @@ -35,7 +35,7 @@ PipelineExecutor::PipelineExecutor( // But for cop/batchCop, there is no such unique identifier, so an empty value is given here, indicating that the query id of PipelineExecutor is invalid. /*query_id=*/context.getDAGContext()->isMPPTask() ? context.getDAGContext()->getMPPTaskId().toString() : "", /*query_id_for_cte=*/context.getDAGContext()->isMPPTask() - ? context.getDAGContext()->getMPPTaskId().getQueryIDForCTE() + ? context.getDAGContext()->getMPPTaskId().getQueryID() : "", req_id, memory_tracker_, diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.h b/dbms/src/Flash/Executor/PipelineExecutorContext.h index 2d8ee77f5ee..c96651c4c41 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.h +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.h @@ -152,7 +152,6 @@ class PipelineExecutorContext : private boost::noncopyable void addCTE(std::shared_ptr cte) { this->dag_context->addCTE(cte); } void addCTE(std::shared_ptr & cte) { this->dag_context->addCTE(cte); } - void sinkNeedRelease() { this->dag_context->sinkNeedRelease(); } void setHasCTESource() { this->dag_context->setHasCTESource(); } private: diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index 9c19702b471..53feba1fdca 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -28,14 +28,18 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id, const St if unlikely (iter == this->ctes.end()) // Maybe the task is cancelled and all ctes have been released return; + auto iter_for_cte = iter->second.find(partition_id); - if unlikely (iter_for_cte == iter->second.end()) - throw Exception(fmt::format("Can't find cte: {}, partition: {}", query_id_and_cte_id, partition_id)); + RUNTIME_CHECK_MSG( + iter_for_cte != iter->second.end(), + "Can't find cte: {}, partition: {}", + query_id_and_cte_id, + partition_id); if (iter_for_cte->second.getTotalExitNum() == iter_for_cte->second.getExpectedTotalNum()) iter->second.erase(iter_for_cte); - if (iter->second.size() == 0) + if (iter->second.empty()) this->ctes.erase(iter); } @@ -84,7 +88,7 @@ void CTEManager::releaseCTEs(const String & query_id_and_cte_id) this->ctes.erase(iter); } -std::shared_ptr CTEManager::getCTEimpl( +std::shared_ptr CTEManager::getCTEImpl( const String & query_id_and_cte_id, const String & partition_id, Int32 expected_sink_num, @@ -93,18 +97,13 @@ std::shared_ptr CTEManager::getCTEimpl( std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if (iter == this->ctes.end()) - this->ctes[query_id_and_cte_id] = std::map{}; + this->ctes[query_id_and_cte_id] = std::unordered_map{}; - auto iter_for_cte = this->ctes[query_id_and_cte_id].find(partition_id); - if (iter_for_cte == this->ctes[query_id_and_cte_id].end()) - { - // It's the first time we request for the specific cte - // Create it because no one created it before. - auto cte = std::make_shared(); - CTEWithCounter cte_with_counter(cte, expected_sink_num, expected_source_num); - this->ctes[query_id_and_cte_id].insert(std::make_pair(partition_id, cte_with_counter)); - return cte; - } + auto & cte_map = this->ctes[query_id_and_cte_id]; + auto [iter_for_cte, _] = cte_map.emplace( + std::piecewise_construct, + std::forward_as_tuple(partition_id), + std::forward_as_tuple(std::make_shared(), expected_sink_num, expected_source_num)); return iter_for_cte->second.getCTE(); } diff --git a/dbms/src/Flash/Mpp/CTEManager.h b/dbms/src/Flash/Mpp/CTEManager.h index 7d9fc2ff5e2..60189390c9c 100644 --- a/dbms/src/Flash/Mpp/CTEManager.h +++ b/dbms/src/Flash/Mpp/CTEManager.h @@ -62,7 +62,7 @@ class CTEManager Int32 expected_sink_num, Int32 expected_source_num) { - return this->getCTEimpl(query_id_and_cte_id, partition_id, expected_sink_num, expected_source_num); + return this->getCTEImpl(query_id_and_cte_id, partition_id, expected_sink_num, expected_source_num); } std::shared_ptr getCTEBySource( const String & query_id_and_cte_id, @@ -70,20 +70,20 @@ class CTEManager Int32 expected_sink_num, Int32 expected_source_num) { - return this->getCTEimpl(query_id_and_cte_id, partition_id, expected_sink_num, expected_source_num); + return this->getCTEImpl(query_id_and_cte_id, partition_id, expected_sink_num, expected_source_num); } void releaseCTEBySource(const String & query_id_and_cte_id, const String & partition_id); void releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id); void releaseCTEs(const String & query_id_and_cte_id); private: - std::shared_ptr getCTEimpl( + std::shared_ptr getCTEImpl( const String & query_id_and_cte_id, const String & partition_id, Int32 expected_sink_num, Int32 expected_source_num); std::mutex mu; - std::map> ctes; + std::unordered_map> ctes; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTaskId.h b/dbms/src/Flash/Mpp/MPPTaskId.h index e3789f04795..8260c20899a 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.h +++ b/dbms/src/Flash/Mpp/MPPTaskId.h @@ -169,7 +169,7 @@ struct MPPTaskId bool isUnknown() const { return task_id == unknown_task_id; } - inline String getQueryIDForCTE() const { return isUnknown() ? "" : gather_id.toStringForQueryID(); } + inline String getQueryID() const { return isUnknown() ? "" : gather_id.toStringForQueryID(); } String toString() const; static const MPPTaskId unknown_mpp_task_id; static const MPPQueryId Max_Query_Id; diff --git a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp index b6722408911..86852ebf86d 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp @@ -40,6 +40,7 @@ WaitReactor::WaitReactor(TaskScheduler & scheduler_) GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_join_probe).Set(0); GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_result_queue_write).Set(0); GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_shared_queue_write).Set(0); + GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte).Set(0); thread = std::thread(&WaitReactor::loop, this); } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index b24bb129edc..d5b9728ba74 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -57,6 +57,9 @@ class PipeConditionVariable case NotifyType::WAIT_ON_SHARED_QUEUE_WRITE: GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_shared_queue_write).Increment(change); break; + case NotifyType::WAIT_ON_CTE: + GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte).Increment(change); + break; } } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h index 809fdc2cfff..3c1d355efb2 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -56,6 +56,7 @@ enum class NotifyType WAIT_ON_JOIN_BUILD_FINISH, WAIT_ON_JOIN_PROBE_FINISH, WAIT_ON_RESULT_QUEUE_WRITE, + WAIT_ON_CTE, }; class PipelineExecutorContext; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 4f636c0a133..9192b67786d 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -64,7 +64,6 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( exec_context.sinkNeedRelease(); } - group_builder.transform([&](auto & builder) { if (fine_grained_shuffle.enabled()) { diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 603e2f0a6dd..2bafecd1ae9 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include @@ -103,6 +103,7 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( void PhysicalCTESource::finalizeImpl(const Names & parent_require) { FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); + FinalizeHelper::prependProjectionInputIfNeed() } const Block & PhysicalCTESource::getSampleBlock() const diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index eddfa0cf458..45c389ab804 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -18,6 +18,8 @@ #include #include +#include "Flash/Pipeline/Schedule/Tasks/Task.h" + namespace DB { CTEOpStatus CTE::tryGetBlockAt(size_t idx, Block & block) @@ -28,12 +30,7 @@ CTEOpStatus CTE::tryGetBlockAt(size_t idx, Block & block) auto block_num = this->blocks.size(); if (block_num <= idx) - { - if (this->is_eof) - return CTEOpStatus::Eof; - else - return CTEOpStatus::Waiting; - } + return this->is_eof ? CTEOpStatus::Eof : CTEOpStatus::Waiting; block = this->blocks[idx]; return CTEOpStatus::Ok; @@ -61,6 +58,7 @@ void CTE::registerTask(TaskPtr && task) std::unique_lock lock(this->rw_lock); if (!this->hasDataNoLock()) { + task->setNotifyType(NotifyType::WAIT_ON_CTE); pipe_cv.registerTask(std::move(task)); return; } diff --git a/dbms/src/Operators/CTESource.cpp b/dbms/src/Operators/CTESourceOp.cpp similarity index 98% rename from dbms/src/Operators/CTESource.cpp rename to dbms/src/Operators/CTESourceOp.cpp index d7cf8e44cab..02fbb8557c3 100644 --- a/dbms/src/Operators/CTESource.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -13,7 +13,7 @@ // limitations under the License. #include -#include +#include #include namespace DB diff --git a/dbms/src/Operators/CTESource.h b/dbms/src/Operators/CTESourceOp.h similarity index 100% rename from dbms/src/Operators/CTESource.h rename to dbms/src/Operators/CTESourceOp.h diff --git a/dbms/src/Operators/Operator.h b/dbms/src/Operators/Operator.h index 83a9a5958a2..5312898848d 100644 --- a/dbms/src/Operators/Operator.h +++ b/dbms/src/Operators/Operator.h @@ -88,7 +88,6 @@ class Operator header = header_; } - const OperatorProfileInfoPtr & getProfileInfo() const { return profile_info_ptr; } virtual IOProfileInfoPtr getIOProfileInfo() const { throw Exception("Unsupport"); } diff --git a/dbms/src/Operators/SharedQueue.cpp b/dbms/src/Operators/SharedQueue.cpp index cf2a163b52f..12ba634d8c8 100644 --- a/dbms/src/Operators/SharedQueue.cpp +++ b/dbms/src/Operators/SharedQueue.cpp @@ -13,7 +13,6 @@ // limitations under the License. #include -#include #include #include From fe3b63da525dba2be932d31aac2caf7ee9d1ece2 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 16 Jun 2025 11:32:07 +0800 Subject: [PATCH 059/118] tweaking --- dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp | 1 - dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp | 1 - dbms/src/Operators/SharedQueue.cpp | 1 + 3 files changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 9192b67786d..c4a74beb523 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -61,7 +61,6 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( { cte = context.getCTEManager() ->getCTEBySink(query_id_and_cte_id, "", this->expected_sink_num, this->expected_source_num); - exec_context.sinkNeedRelease(); } group_builder.transform([&](auto & builder) { diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 2bafecd1ae9..2d426c6dbc8 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -103,7 +103,6 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( void PhysicalCTESource::finalizeImpl(const Names & parent_require) { FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); - FinalizeHelper::prependProjectionInputIfNeed() } const Block & PhysicalCTESource::getSampleBlock() const diff --git a/dbms/src/Operators/SharedQueue.cpp b/dbms/src/Operators/SharedQueue.cpp index 12ba634d8c8..cf2a163b52f 100644 --- a/dbms/src/Operators/SharedQueue.cpp +++ b/dbms/src/Operators/SharedQueue.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include From 381944cf42f765576c556f3232318e2f23a7ad79 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 16 Jun 2025 16:33:15 +0800 Subject: [PATCH 060/118] minor fix --- dbms/src/Flash/Mpp/MPPTask.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index a105970b5ea..44884306086 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -573,12 +573,15 @@ void MPPTask::runImpl() auto time_cost_in_preprocess_ms = time_cost_in_preprocess_ns / MILLISECOND_TO_NANO; LOG_DEBUG(log, "task preprocess done"); schedule_entry.setNeededThreads(estimateCountOfNewThreads()); - LOG_DEBUG( - log, - "Estimate new thread count of query: {} including tunnel_threads: {}, receiver_threads: {}", - schedule_entry.getNeededThreads(), - dag_context->tunnel_set->getExternalThreadCnt(), - new_thread_count_of_mpp_receiver); + + // tunnel_set may be nullptr when we get cte sink + if (dag_context->tunnel_set != nullptr) + LOG_DEBUG( + log, + "Estimate new thread count of query: {} including tunnel_threads: {}, receiver_threads: {}", + schedule_entry.getNeededThreads(), + dag_context->tunnel_set->getExternalThreadCnt(), + new_thread_count_of_mpp_receiver); scheduleOrWait(); From 51ab8c1a22f8646515e3ff13fdd397992e04d8c2 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 17 Jun 2025 13:50:28 +0800 Subject: [PATCH 061/118] tweaking --- dbms/src/Common/TiFlashMetrics.h | 3 +- .../Schedule/Tasks/PipeConditionVariable.h | 7 ++--- dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 3 +- dbms/src/Operators/CTE.cpp | 30 +++++++++---------- 4 files changed, 19 insertions(+), 24 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 9f3a624b4fe..7167e75e205 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -771,8 +771,7 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_wait_on_join_build, {"type", "wait_on_join_build"}), \ F(type_wait_on_join_probe, {"type", "wait_on_join_probe"}), \ F(type_wait_on_result_queue_write, {"type", "wait_on_result_queue_write"}), \ - F(type_wait_on_cte_sink, {"type", "wait_on_cte_sink"}), \ - F(type_wait_on_cte_source, {"type", "wait_on_cte_source"})) \ + F(type_wait_on_cte, {"type", "wait_on_cte"})) \ M(tiflash_pipeline_task_duration_seconds, \ "Bucketed histogram of pipeline task duration in seconds", \ Histogram, /* these command usually cost several hundred milliseconds to several seconds, increase the start bucket to 5ms */ \ diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index 6393d53ad5a..c14b23ffd7d 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -57,11 +57,8 @@ class PipeConditionVariable case NotifyType::WAIT_ON_SHARED_QUEUE_WRITE: GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_shared_queue_write).Increment(change); break; - case NotifyType::WAIT_ON_CTE_SINK: - GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte_sink); - break; - case NotifyType::WAIT_ON_CTE_SOURCE: - GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte_source); + case NotifyType::WAIT_ON_CTE: + GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte); break; } } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h index 1205cc84773..3c1d355efb2 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -56,8 +56,7 @@ enum class NotifyType WAIT_ON_JOIN_BUILD_FINISH, WAIT_ON_JOIN_PROBE_FINISH, WAIT_ON_RESULT_QUEUE_WRITE, - WAIT_ON_CTE_SINK, - WAIT_ON_CTE_SOURCE, + WAIT_ON_CTE, }; class PipelineExecutorContext; diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 6e3981a45e7..7ab282fa1da 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -54,44 +54,44 @@ CTEOpStatus CTE::tryGetBlockAt(size_t idx, Block & block) return CTEOpStatus::Ok; } -Status CTE::checkAvailableBlock(size_t idx) +CTEOpStatus CTE::checkAvailableBlock(size_t idx) { { std::shared_lock status_lock(this->aux_rw_lock); if (this->cte_status != CTE::Normal) - return Status::BlockUnavailable; + return CTEOpStatus::BlockUnavailable; } std::shared_lock lock(this->rw_lock); if unlikely (this->is_cancelled) - return Status::Cancelled; + return CTEOpStatus::Cancelled; if (this->is_spill_triggered) { auto spilled_block_num = static_cast(this->cte_spill.blockNum()); if (idx < spilled_block_num) - return Status::Ok; + return CTEOpStatus::Ok; idx -= spilled_block_num; } if (this->blocks.size() > idx) - return Status::Ok; - return Status::BlockUnavailable; + return CTEOpStatus::Ok; + return CTEOpStatus::BlockUnavailable; } -Status CTE::pushBlock(const Block & block) +CTEOpStatus CTE::pushBlock(const Block & block) { std::unique_lock lock(this->rw_lock, std::defer_lock); - Status ret = Status::Ok; + CTEOpStatus ret = CTEOpStatus::Ok; { std::unique_lock status_lock(this->aux_rw_lock); if (this->cte_status != CTE::Normal) { // Block memory usage will be calculated after the finish of spill this->tmp_blocks.push_back(block); - return Status::IOOut; + return CTEOpStatus::IOOut; } // This function is called in cpu pool, we don't want to wait for this lock too long. @@ -99,17 +99,17 @@ Status CTE::pushBlock(const Block & block) lock.lock(); if unlikely (this->is_cancelled) - return Status::Cancelled; + return CTEOpStatus::Cancelled; if unlikely (block.rows() == 0) // All rows in block may have been filtered and it's needles to store this block - return Status::Ok; + return CTEOpStatus::Ok; this->memory_usage += block.bytes(); if (this->memory_usage >= this->memory_threshold) { this->cte_status = CTE::NeedSpill; - ret = Status::IOOut; + ret = CTEOpStatus::IOOut; } } @@ -120,11 +120,11 @@ Status CTE::pushBlock(const Block & block) return ret; } -Status CTE::getBlockFromDisk(size_t idx, Block & block) +CTEOpStatus CTE::getBlockFromDisk(size_t idx, Block & block) { std::shared_lock lock(this->rw_lock); if unlikely (this->is_cancelled) - return Status::Cancelled; + return CTEOpStatus::Cancelled; if unlikely (!this->is_spill_triggered) // We can call this function only when spill is triggered @@ -134,7 +134,7 @@ Status CTE::getBlockFromDisk(size_t idx, Block & block) throw Exception("Requested block is not in disk"); block = this->cte_spill.readBlockAt(idx); - return Status::Ok; + return CTEOpStatus::Ok; } bool CTE::spillBlocks() From cce3d0c825d6a2c502dcea55de89046649f9d152 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 17 Jun 2025 18:24:51 +0800 Subject: [PATCH 062/118] change WAITING to WAITING_FOR_NOTIFY --- dbms/src/Operators/CTE.cpp | 40 ++++++++++++++--------------- dbms/src/Operators/CTE.h | 27 ++++++++++++++------ dbms/src/Operators/CTEReader.cpp | 21 ++------------- dbms/src/Operators/CTEReader.h | 41 ++++++++++++++++++++++-------- dbms/src/Operators/CTESourceOp.cpp | 30 +++------------------- dbms/src/Operators/CTESourceOp.h | 2 -- 6 files changed, 75 insertions(+), 86 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 45c389ab804..e42d8adc078 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -12,28 +12,24 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include #include -#include "Flash/Pipeline/Schedule/Tasks/Task.h" - namespace DB { CTEOpStatus CTE::tryGetBlockAt(size_t idx, Block & block) { std::shared_lock lock(this->rw_lock); - if unlikely (this->is_cancelled) - return CTEOpStatus::Cancelled; - - auto block_num = this->blocks.size(); - if (block_num <= idx) - return this->is_eof ? CTEOpStatus::Eof : CTEOpStatus::Waiting; + auto status = this->checkBlockAvailableNoLock(idx); + if (status != CTEOpStatus::Ok) + return status; block = this->blocks[idx]; - return CTEOpStatus::Ok; + return status; } bool CTE::pushBlock(const Block & block) @@ -46,23 +42,27 @@ bool CTE::pushBlock(const Block & block) return true; this->memory_usage += block.bytes(); - if unlikely (this->blocks.empty()) - this->pipe_cv.notifyAll(); this->blocks.push_back(block); + this->pipe_cv.notifyOne(); return true; } -void CTE::registerTask(TaskPtr && task) +void CTE::registerTask(TaskPtr && task, NotifyType type) { + task->setNotifyType(type); + pipe_cv.registerTask(std::move(task)); +} + +void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t expected_block_fetch_idx) +{ + std::shared_lock shared_lock(this->rw_lock); + CTEOpStatus status = this->checkBlockAvailableNoLock(expected_block_fetch_idx); + if (status == CTEOpStatus::Ok) { - std::unique_lock lock(this->rw_lock); - if (!this->hasDataNoLock()) - { - task->setNotifyType(NotifyType::WAIT_ON_CTE); - pipe_cv.registerTask(std::move(task)); - return; - } + this->notifyTaskDirectly(std::move(task)); + return; } - this->pipe_cv.notifyTaskDirectly(std::move(task)); + + this->registerTask(std::move(task), NotifyType::WAIT_ON_CTE); } } // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index ebaecf5a1f4..2de951a9233 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -28,20 +29,23 @@ namespace DB enum class CTEOpStatus { Ok, - Waiting, + BlockNotAvailable, Eof, Cancelled }; -class CTE : public NotifyFuture +class CTE { public: CTEOpStatus tryGetBlockAt(size_t idx, Block & block); + bool pushBlock(const Block & block); void notifyEOF() { this->notifyImpl(true); } void notifyCancel() { this->notifyImpl(false); } - void registerTask(TaskPtr && task) override; + void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t expected_block_fetch_idx); + void registerTask(TaskPtr && task, NotifyType type); + void notifyTaskDirectly(TaskPtr && task) { this->pipe_cv.notifyTaskDirectly(std::move(task)); } void addResp(const tipb::SelectResponse & resp) { @@ -59,6 +63,18 @@ class CTE : public NotifyFuture } private: + CTEOpStatus checkBlockAvailableNoLock(size_t idx) + { + if unlikely (this->is_cancelled) + return CTEOpStatus::Cancelled; + + auto block_num = this->blocks.size(); + if (block_num <= idx) + return this->is_eof ? CTEOpStatus::Eof : CTEOpStatus::BlockNotAvailable; + + return CTEOpStatus::Ok; + } + template void notifyImpl(bool is_eof) { @@ -75,15 +91,10 @@ class CTE : public NotifyFuture this->pipe_cv.notifyAll(); } - // Return true if CTE has data - inline bool hasDataNoLock() const { return !this->blocks.empty(); } - std::shared_mutex rw_lock; Blocks blocks; size_t memory_usage = 0; - // Tasks in WAITING_FOR_NOTIFY are saved in this deque - std::deque waiting_tasks; PipeConditionVariable pipe_cv; bool is_eof = false; diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 510074ec132..21d0864a05c 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -29,7 +29,8 @@ std::pair CTEReader::fetchNextBlock() case CTEOpStatus::Eof: if (this->resp.execution_summaries_size() == 0) this->cte->tryToGetResp(this->resp); - case CTEOpStatus::Waiting: + case CTEOpStatus::BlockNotAvailable: + this->notifier.setFetchBlockIdx(this->block_fetch_idx); case CTEOpStatus::Cancelled: return {ret, Block()}; case CTEOpStatus::Ok: @@ -38,22 +39,4 @@ std::pair CTEReader::fetchNextBlock() } throw Exception("Should not reach here"); } - -CTEOpStatus CTEReader::checkAvailableBlock() -{ - Block block; - std::lock_guard lock(this->mu); - auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx, block); - switch (ret) - { - case CTEOpStatus::Eof: - case CTEOpStatus::Waiting: - case CTEOpStatus::Cancelled: - return ret; - case CTEOpStatus::Ok: - // Do not add block_fetch_idx here as we only check if there are available blocks - return CTEOpStatus::Ok; - } - throw Exception("Should not reach here"); -} } // namespace DB diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index d204c5a2b5f..c371bd5ddf1 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -16,14 +16,42 @@ #include #include +#include #include #include #include #include +#include namespace DB { +class CTEReaderNotifyFuture : public NotifyFuture +{ +public: + explicit CTEReaderNotifyFuture(std::shared_ptr cte_) + : cte(cte_) + {} + + void setFetchBlockIdx(size_t idx) + { + std::unique_lock lock(this->rw_lock); + expected_block_fetch_idx = idx; + } + + void registerTask(TaskPtr && task) override + { + std::shared_lock shared_lock(this->rw_lock); + this->cte->checkBlockAvailableAndRegisterTask(std::move(task), this->expected_block_fetch_idx); + } + +private: + std::shared_mutex rw_lock; + size_t expected_block_fetch_idx = 0; + + std::shared_ptr cte; +}; + class CTEReader { public: @@ -38,6 +66,7 @@ class CTEReader , cte_manager(cte_manager_) , cte(cte_manager_ ->getCTEBySource(query_id_and_cte_id_, partition_id, expected_sink_num_, expected_source_num_)) + , notifier(cte) {} ~CTEReader() @@ -58,16 +87,7 @@ class CTEReader resp.CopyFrom(this->resp); } - bool isBlockGenerated() - { - std::lock_guard lock(this->mu); - - // `block_fetch_idx == 0` means that CTE hasn't received block yet, maybe it is waiting - // for the finish of join executor and etc. - return this->block_fetch_idx != 0; - } - - void setNotifyFuture() { ::DB::setNotifyFuture(cte.get()); } + void setNotifyFuture() { ::DB::setNotifyFuture(&(this->notifier)); } std::shared_ptr getCTE() const { return this->cte; } @@ -76,6 +96,7 @@ class CTEReader String partition_id; CTEManager * cte_manager; std::shared_ptr cte; + CTEReaderNotifyFuture notifier; std::mutex mu; size_t block_fetch_idx = 0; diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index 02fbb8557c3..a8e3190a51e 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -36,33 +36,9 @@ OperatorStatus CTESourceOp::readImpl(Block & block) block = res.second; this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; - case CTEOpStatus::Waiting: - if likely (this->cte_reader->isBlockGenerated()) - { - return OperatorStatus::WAITING; - } - else - { - // CTE has not begun to receive data yet - // So we need to wait the notify from CTE - this->cte_reader->setNotifyFuture(); - return OperatorStatus::WAIT_FOR_NOTIFY; - } - case CTEOpStatus::Cancelled: - return OperatorStatus::CANCELLED; - } -} - -OperatorStatus CTESourceOp::awaitImpl() -{ - auto res = this->cte_reader->checkAvailableBlock(); - switch (res) - { - case CTEOpStatus::Eof: - case CTEOpStatus::Ok: - return OperatorStatus::HAS_OUTPUT; - case CTEOpStatus::Waiting: - return OperatorStatus::WAITING; + case CTEOpStatus::BlockNotAvailable: + this->cte_reader->setNotifyFuture(); + return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::Cancelled: return OperatorStatus::CANCELLED; } diff --git a/dbms/src/Operators/CTESourceOp.h b/dbms/src/Operators/CTESourceOp.h index 1762e506024..0f041fd76f8 100644 --- a/dbms/src/Operators/CTESourceOp.h +++ b/dbms/src/Operators/CTESourceOp.h @@ -50,8 +50,6 @@ class CTESourceOp : public SourceOp OperatorStatus readImpl(Block & block) override; - OperatorStatus awaitImpl() override; - private: std::shared_ptr cte_reader; uint64_t total_rows{}; From ca61bd21db62725e91c151c76af28c72f2675aaa Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 19 Jun 2025 18:25:48 +0800 Subject: [PATCH 063/118] pass error message by cte --- .../Flash/Executor/PipelineExecutorContext.cpp | 5 +++++ dbms/src/Operators/CTE.h | 17 ++++++++++++++++- dbms/src/Operators/CTEReader.cpp | 2 ++ dbms/src/Operators/CTESinkOp.cpp | 8 +++++--- dbms/src/Operators/CTESourceOp.cpp | 2 ++ 5 files changed, 30 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.cpp b/dbms/src/Flash/Executor/PipelineExecutorContext.cpp index 32c1ca37e4c..2d14a38a8f3 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.cpp @@ -185,6 +185,11 @@ void PipelineExecutorContext::cancel() // pipeline tasks waiting in the WAIT_FOR_NOTIFY state from never being notified. if (dag_context->tunnel_set) dag_context->tunnel_set->close(getTrimmedErrMsg(), false); + + auto ctes = dag_context->getCTEs(); + if (!ctes.empty()) + ctes[0]->notifyError(getTrimmedErrMsg()); + if (auto mpp_receiver_set = dag_context->getMPPReceiverSet(); mpp_receiver_set) mpp_receiver_set->cancel(); } diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 2de951a9233..8f1b85a44d9 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -31,7 +31,8 @@ enum class CTEOpStatus Ok, BlockNotAvailable, Eof, - Cancelled + Cancelled, + Error }; class CTE @@ -43,6 +44,18 @@ class CTE void notifyEOF() { this->notifyImpl(true); } void notifyCancel() { this->notifyImpl(false); } + void notifyError(const String & err_msg) + { + std::unique_lock lock(this->rw_lock); + this->err_msg = err_msg; + } + + String getError() + { + std::shared_lock lock(this->rw_lock); + return this->err_msg; + } + void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t expected_block_fetch_idx); void registerTask(TaskPtr && task, NotifyType type); void notifyTaskDirectly(TaskPtr && task) { this->pipe_cv.notifyTaskDirectly(std::move(task)); } @@ -102,5 +115,7 @@ class CTE bool get_resp = false; tipb::SelectResponse resp; + + String err_msg; }; } // namespace DB diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 21d0864a05c..0aa84ac7ba8 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -36,6 +36,8 @@ std::pair CTEReader::fetchNextBlock() case CTEOpStatus::Ok: this->block_fetch_idx++; return {ret, block}; + case DB::CTEOpStatus::Error: + throw Exception(this->cte->getError()); } throw Exception("Should not reach here"); } diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 12c28a456ba..86b9c8a9f40 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -25,12 +25,14 @@ void CTESinkOp::operateSuffixImpl() OperatorStatus CTESinkOp::writeImpl(Block && block) { if (!block) - { return OperatorStatus::FINISHED; - } + this->total_rows += block.rows(); if (this->cte->pushBlock(block)) return OperatorStatus::NEED_INPUT; - return OperatorStatus::CANCELLED; + const String & err = this->cte->getError(); + if (err.empty()) + return OperatorStatus::CANCELLED; + throw Exception(err); } } // namespace DB diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index a8e3190a51e..69027f36a56 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -41,6 +41,8 @@ OperatorStatus CTESourceOp::readImpl(Block & block) return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::Cancelled: return OperatorStatus::CANCELLED; + default: + throw Exception("Should not reach here"); } } } // namespace DB From e68baa209d47cc0f8f187c818010505c674b82b9 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 20 Jun 2025 10:12:52 +0800 Subject: [PATCH 064/118] refine --- dbms/src/Operators/CTE.cpp | 21 ++++++----- dbms/src/Operators/CTE.h | 56 ++++++++++++++++++++++++------ dbms/src/Operators/CTEReader.cpp | 14 ++++---- dbms/src/Operators/CTEReader.h | 25 +++++-------- dbms/src/Operators/CTESinkOp.h | 2 ++ dbms/src/Operators/CTESourceOp.cpp | 5 ++- dbms/src/Operators/CTESourceOp.h | 1 - 7 files changed, 75 insertions(+), 49 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index e42d8adc078..e4edc7da76e 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -17,24 +17,26 @@ #include #include -#include namespace DB { -CTEOpStatus CTE::tryGetBlockAt(size_t idx, Block & block) +CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, Block & block) { - std::shared_lock lock(this->rw_lock); - auto status = this->checkBlockAvailableNoLock(idx); + std::lock_guard read_lock(this->read_mu); + std::lock_guard lock(this->mu); + auto status = this->checkBlockAvailableNoLock(cte_reader_id); if (status != CTEOpStatus::Ok) return status; - block = this->blocks[idx]; + block = this->blocks[this->fetch_block_idxs[cte_reader_id].idx]; + this->fetch_block_idxs[cte_reader_id].idx++; return status; } bool CTE::pushBlock(const Block & block) { - std::unique_lock lock(this->rw_lock); + std::lock_guard write_lock(this->write_mu); + std::lock_guard lock(this->mu); if unlikely (this->is_cancelled) return false; @@ -53,10 +55,11 @@ void CTE::registerTask(TaskPtr && task, NotifyType type) pipe_cv.registerTask(std::move(task)); } -void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t expected_block_fetch_idx) +void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id) { - std::shared_lock shared_lock(this->rw_lock); - CTEOpStatus status = this->checkBlockAvailableNoLock(expected_block_fetch_idx); + std::lock_guard lock(this->read_mu); + std::lock_guard shared_lock(this->mu); + CTEOpStatus status = this->checkBlockAvailableNoLock(cte_reader_id); if (status == CTEOpStatus::Ok) { this->notifyTaskDirectly(std::move(task)); diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 8f1b85a44d9..1c8fa6ab236 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -19,10 +19,11 @@ #include #include #include +#include #include #include -#include +#include namespace DB { @@ -35,10 +36,31 @@ enum class CTEOpStatus Error }; +struct IdxWithPadding +{ + explicit IdxWithPadding(size_t idx_) + : idx(idx_) + {} + + size_t idx; + + // To avoid false sharing + char padding[ABSL_CACHELINE_SIZE]{}; +}; + class CTE { public: - CTEOpStatus tryGetBlockAt(size_t idx, Block & block); + size_t getCTEReaderID() + { + std::lock_guard lock(this->mu); + auto ret = this->next_cte_reader_id; + this->next_cte_reader_id++; + this->fetch_block_idxs.insert(std::make_tuple(ret, IdxWithPadding(0))); + return ret; + } + + CTEOpStatus tryGetBlockAt(size_t cte_reader_id, Block & block); bool pushBlock(const Block & block); void notifyEOF() { this->notifyImpl(true); } @@ -46,28 +68,30 @@ class CTE void notifyError(const String & err_msg) { - std::unique_lock lock(this->rw_lock); + std::lock_guard lock(this->mu); this->err_msg = err_msg; } String getError() { - std::shared_lock lock(this->rw_lock); + std::lock_guard lock(this->mu); return this->err_msg; } - void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t expected_block_fetch_idx); + void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id); + void registerTask(TaskPtr && task, NotifyType type); void notifyTaskDirectly(TaskPtr && task) { this->pipe_cv.notifyTaskDirectly(std::move(task)); } void addResp(const tipb::SelectResponse & resp) { - std::unique_lock lock(this->rw_lock); + std::lock_guard lock(this->mu); this->resp.MergeFrom(resp); } void tryToGetResp(tipb::SelectResponse & resp) { + std::lock_guard lock(this->mu); if (!this->get_resp) { this->get_resp = true; @@ -76,13 +100,12 @@ class CTE } private: - CTEOpStatus checkBlockAvailableNoLock(size_t idx) + CTEOpStatus checkBlockAvailableNoLock(size_t cte_reader_id) { if unlikely (this->is_cancelled) return CTEOpStatus::Cancelled; - auto block_num = this->blocks.size(); - if (block_num <= idx) + if (this->blocks.size() <= this->fetch_block_idxs[cte_reader_id].idx) return this->is_eof ? CTEOpStatus::Eof : CTEOpStatus::BlockNotAvailable; return CTEOpStatus::Ok; @@ -91,7 +114,7 @@ class CTE template void notifyImpl(bool is_eof) { - std::unique_lock lock(this->rw_lock, std::defer_lock); + std::unique_lock lock(this->mu, std::defer_lock); if constexpr (has_lock) lock.lock(); @@ -104,9 +127,20 @@ class CTE this->pipe_cv.notifyAll(); } - std::shared_mutex rw_lock; + // Suppose there are 100 read threads and 10 write threads, for write threads + // they are hard to get mu lock and the write operation will be blocked + // for longer time. In order to make read operation and write operation to + // have same opportunity to get the mu, we introduce read_mu and write_mu, + // so that there are only one read thread and one write thread to try to + // get the mu. + std::mutex read_mu; + std::mutex write_mu; + + std::mutex mu; Blocks blocks; size_t memory_usage = 0; + std::unordered_map fetch_block_idxs; + size_t next_cte_reader_id = 0; PipeConditionVariable pipe_cv; diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 0aa84ac7ba8..9bdc4094c67 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -19,23 +19,21 @@ namespace DB { -std::pair CTEReader::fetchNextBlock() +CTEOpStatus CTEReader::fetchNextBlock(Block & block) { - Block block; - std::lock_guard lock(this->mu); - auto ret = this->cte->tryGetBlockAt(this->block_fetch_idx, block); + auto ret = this->cte->tryGetBlockAt(this->cte_reader_id, block); switch (ret) { case CTEOpStatus::Eof: + { + std::lock_guard lock(this->mu); if (this->resp.execution_summaries_size() == 0) this->cte->tryToGetResp(this->resp); + } case CTEOpStatus::BlockNotAvailable: - this->notifier.setFetchBlockIdx(this->block_fetch_idx); case CTEOpStatus::Cancelled: - return {ret, Block()}; case CTEOpStatus::Ok: - this->block_fetch_idx++; - return {ret, block}; + return ret; case DB::CTEOpStatus::Error: throw Exception(this->cte->getError()); } diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index c371bd5ddf1..be8a1f1c396 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -22,34 +22,25 @@ #include #include -#include namespace DB { class CTEReaderNotifyFuture : public NotifyFuture { public: - explicit CTEReaderNotifyFuture(std::shared_ptr cte_) + CTEReaderNotifyFuture(std::shared_ptr cte_, size_t cte_reader_id_) : cte(cte_) + , cte_reader_id(cte_reader_id_) {} - void setFetchBlockIdx(size_t idx) - { - std::unique_lock lock(this->rw_lock); - expected_block_fetch_idx = idx; - } - void registerTask(TaskPtr && task) override { - std::shared_lock shared_lock(this->rw_lock); - this->cte->checkBlockAvailableAndRegisterTask(std::move(task), this->expected_block_fetch_idx); + this->cte->checkBlockAvailableAndRegisterTask(std::move(task), this->cte_reader_id); } private: - std::shared_mutex rw_lock; - size_t expected_block_fetch_idx = 0; - std::shared_ptr cte; + size_t cte_reader_id; }; class CTEReader @@ -66,7 +57,8 @@ class CTEReader , cte_manager(cte_manager_) , cte(cte_manager_ ->getCTEBySource(query_id_and_cte_id_, partition_id, expected_sink_num_, expected_source_num_)) - , notifier(cte) + , cte_reader_id(this->cte->getCTEReaderID()) + , notifier(cte, this->cte_reader_id) {} ~CTEReader() @@ -75,7 +67,7 @@ class CTEReader this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id, this->partition_id); } - std::pair fetchNextBlock(); + CTEOpStatus fetchNextBlock(Block & block); CTEOpStatus checkAvailableBlock(); void getResp(tipb::SelectResponse & resp) @@ -96,11 +88,10 @@ class CTEReader String partition_id; CTEManager * cte_manager; std::shared_ptr cte; + size_t cte_reader_id; CTEReaderNotifyFuture notifier; std::mutex mu; - size_t block_fetch_idx = 0; - bool resp_fetched = false; tipb::SelectResponse resp; }; diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 10bb689e223..e1a8afb9474 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -18,6 +18,8 @@ #include #include +#include "Common/Stopwatch.h" + namespace DB { class CTESinkOp : public SinkOp diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index 69027f36a56..4fcf2b17eb0 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -25,15 +25,14 @@ void CTESourceOp::operateSuffixImpl() OperatorStatus CTESourceOp::readImpl(Block & block) { - auto res = this->cte_reader->fetchNextBlock(); - switch (res.first) + auto ret = this->cte_reader->fetchNextBlock(block); + switch (ret) { case CTEOpStatus::Eof: this->cte_reader->getResp(this->resp); if (this->resp.execution_summaries_size() != 0) this->io_profile_info->remote_execution_summary.add(this->resp); case CTEOpStatus::Ok: - block = res.second; this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; case CTEOpStatus::BlockNotAvailable: diff --git a/dbms/src/Operators/CTESourceOp.h b/dbms/src/Operators/CTESourceOp.h index 0f041fd76f8..8959be5c06b 100644 --- a/dbms/src/Operators/CTESourceOp.h +++ b/dbms/src/Operators/CTESourceOp.h @@ -42,7 +42,6 @@ class CTESourceOp : public SourceOp } String getName() const override { return "CTESourceOp"; } - IOProfileInfoPtr getIOProfileInfo() const override { return io_profile_info; } protected: From 53e4ea8cf3eca3d1b670353fef846789657e57c0 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 20 Jun 2025 19:00:21 +0800 Subject: [PATCH 065/118] add concurrency for cte --- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 5 +- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 6 +- dbms/src/Operators/CTE.cpp | 63 ++++++++++++----- dbms/src/Operators/CTE.h | 67 ++++++++++--------- dbms/src/Operators/CTEReader.cpp | 4 +- dbms/src/Operators/CTEReader.h | 25 +------ dbms/src/Operators/CTESinkOp.cpp | 2 +- dbms/src/Operators/CTESinkOp.h | 6 +- dbms/src/Operators/CTESourceOp.cpp | 5 +- dbms/src/Operators/CTESourceOp.h | 25 +++++++ 10 files changed, 125 insertions(+), 83 deletions(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index c4a74beb523..ce43100635d 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -63,6 +63,8 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( ->getCTEBySink(query_id_and_cte_id, "", this->expected_sink_num, this->expected_source_num); } + size_t id = 0; + group_builder.transform([&](auto & builder) { if (fine_grained_shuffle.enabled()) { @@ -74,7 +76,8 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( partition_id++; } - builder.setSinkOp(std::make_unique(exec_context, log->identifier(), cte)); + builder.setSinkOp(std::make_unique(exec_context, log->identifier(), cte, id)); + id++; }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 2d426c6dbc8..2c24bb9402a 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -78,7 +78,7 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( this->expected_source_num); exec_context.addCTE(cte_reader->getCTE()); group_builder.addConcurrency( - std::make_unique(exec_context, log->identifier(), cte_reader, schema)); + std::make_unique(exec_context, log->identifier(), cte_reader, partition_id, schema)); } } else @@ -90,10 +90,10 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( this->expected_sink_num, this->expected_source_num); exec_context.addCTE(cte_reader->getCTE()); - for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) + for (size_t i = 0; i < concurrency; ++i) { group_builder.addConcurrency( - std::make_unique(exec_context, log->identifier(), cte_reader, schema)); + std::make_unique(exec_context, log->identifier(), cte_reader, i, schema)); } } diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index e4edc7da76e..ce246984ba8 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -17,34 +17,57 @@ #include #include +#include namespace DB { -CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, Block & block) +constexpr size_t PARTITION_NUM = 70; // TODO maybe need more tests to select a reasonable value + +inline size_t getPartitionID(size_t id) { return id % PARTITION_NUM; } + +CTE::CTE() { - std::lock_guard read_lock(this->read_mu); - std::lock_guard lock(this->mu); - auto status = this->checkBlockAvailableNoLock(cte_reader_id); + for (size_t i = 0; i < PARTITION_NUM; i++) + { + this->partitions.push_back(CTEPartition()); + this->partitions.back().mu = std::make_unique(); + this->partitions.back().read_mu = std::make_unique(); + this->partitions.back().write_mu = std::make_unique(); + } +} + +CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & block) +{ + auto partition_id = getPartitionID(source_id); + + std::shared_lock rw_lock(this->rw_lock); + std::lock_guard read_lock(*this->partitions[partition_id].read_mu); + std::lock_guard lock(*this->partitions[partition_id].mu); + auto status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); if (status != CTEOpStatus::Ok) return status; - block = this->blocks[this->fetch_block_idxs[cte_reader_id].idx]; - this->fetch_block_idxs[cte_reader_id].idx++; + auto idx = this->partitions[partition_id].fetch_block_idxs[cte_reader_id].idx++; + block = this->partitions[partition_id].blocks[idx]; return status; } -bool CTE::pushBlock(const Block & block) +bool CTE::pushBlock(size_t sink_id, const Block & block) { - std::lock_guard write_lock(this->write_mu); - std::lock_guard lock(this->mu); - if unlikely (this->is_cancelled) - return false; + { + std::shared_lock rw_lock(this->rw_lock); + if unlikely (this->is_cancelled) + return false; + } if unlikely (block.rows() == 0) return true; - this->memory_usage += block.bytes(); - this->blocks.push_back(block); + auto partition_id = getPartitionID(sink_id); + std::lock_guard write_lock(*this->partitions[partition_id].write_mu); + std::lock_guard lock(*this->partitions[partition_id].mu); + this->partitions[partition_id].memory_usages += block.bytes(); + this->partitions[partition_id].blocks.push_back(block); this->pipe_cv.notifyOne(); return true; } @@ -55,11 +78,17 @@ void CTE::registerTask(TaskPtr && task, NotifyType type) pipe_cv.registerTask(std::move(task)); } -void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id) +void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t source_id) { - std::lock_guard lock(this->read_mu); - std::lock_guard shared_lock(this->mu); - CTEOpStatus status = this->checkBlockAvailableNoLock(cte_reader_id); + auto partition_id = getPartitionID(source_id); + CTEOpStatus status; + + { + std::shared_lock rw_lock(this->rw_lock); + std::lock_guard lock(*this->partitions[partition_id].mu); + status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); + } + if (status == CTEOpStatus::Ok) { this->notifyTaskDirectly(std::move(task)); diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 1c8fa6ab236..4c377b3c5d1 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -22,8 +22,11 @@ #include #include +#include #include +#include #include +#include namespace DB { @@ -38,60 +41,74 @@ enum class CTEOpStatus struct IdxWithPadding { + IdxWithPadding() = default; explicit IdxWithPadding(size_t idx_) : idx(idx_) {} - size_t idx; + size_t idx = 0; // To avoid false sharing char padding[ABSL_CACHELINE_SIZE]{}; }; +struct CTEPartition +{ + std::unique_ptr mu; + std::unique_ptr read_mu; + std::unique_ptr write_mu; + Blocks blocks; + std::unordered_map fetch_block_idxs; + size_t memory_usages = 0; // TODO need a unified statistic +}; + class CTE { public: + CTE(); + size_t getCTEReaderID() { - std::lock_guard lock(this->mu); - auto ret = this->next_cte_reader_id; + std::unique_lock lock(this->rw_lock); + auto cte_reader_id = this->next_cte_reader_id; this->next_cte_reader_id++; - this->fetch_block_idxs.insert(std::make_tuple(ret, IdxWithPadding(0))); - return ret; + for (auto & item : this->partitions) + item.fetch_block_idxs.insert(std::make_pair(cte_reader_id, IdxWithPadding(0))); + return cte_reader_id; } - CTEOpStatus tryGetBlockAt(size_t cte_reader_id, Block & block); + CTEOpStatus tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & block); - bool pushBlock(const Block & block); + bool pushBlock(size_t sink_id, const Block & block); void notifyEOF() { this->notifyImpl(true); } void notifyCancel() { this->notifyImpl(false); } void notifyError(const String & err_msg) { - std::lock_guard lock(this->mu); + std::unique_lock lock(this->rw_lock); this->err_msg = err_msg; } String getError() { - std::lock_guard lock(this->mu); + std::shared_lock lock(this->rw_lock); return this->err_msg; } - void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id); + void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t source_id); void registerTask(TaskPtr && task, NotifyType type); void notifyTaskDirectly(TaskPtr && task) { this->pipe_cv.notifyTaskDirectly(std::move(task)); } void addResp(const tipb::SelectResponse & resp) { - std::lock_guard lock(this->mu); + std::unique_lock lock(this->rw_lock); this->resp.MergeFrom(resp); } void tryToGetResp(tipb::SelectResponse & resp) { - std::lock_guard lock(this->mu); + std::shared_lock lock(this->rw_lock); if (!this->get_resp) { this->get_resp = true; @@ -100,12 +117,12 @@ class CTE } private: - CTEOpStatus checkBlockAvailableNoLock(size_t cte_reader_id) + CTEOpStatus checkBlockAvailableNoLock(size_t cte_reader_id, size_t partition_id) { if unlikely (this->is_cancelled) return CTEOpStatus::Cancelled; - if (this->blocks.size() <= this->fetch_block_idxs[cte_reader_id].idx) + if (this->partitions[partition_id].blocks.size() <= this->partitions[partition_id].fetch_block_idxs[cte_reader_id].idx) return this->is_eof ? CTEOpStatus::Eof : CTEOpStatus::BlockNotAvailable; return CTEOpStatus::Ok; @@ -114,7 +131,7 @@ class CTE template void notifyImpl(bool is_eof) { - std::unique_lock lock(this->mu, std::defer_lock); + std::unique_lock lock(this->rw_lock, std::defer_lock); if constexpr (has_lock) lock.lock(); @@ -127,23 +144,11 @@ class CTE this->pipe_cv.notifyAll(); } - // Suppose there are 100 read threads and 10 write threads, for write threads - // they are hard to get mu lock and the write operation will be blocked - // for longer time. In order to make read operation and write operation to - // have same opportunity to get the mu, we introduce read_mu and write_mu, - // so that there are only one read thread and one write thread to try to - // get the mu. - std::mutex read_mu; - std::mutex write_mu; - - std::mutex mu; - Blocks blocks; - size_t memory_usage = 0; - std::unordered_map fetch_block_idxs; - size_t next_cte_reader_id = 0; - + std::vector partitions; PipeConditionVariable pipe_cv; - + + std::shared_mutex rw_lock; + size_t next_cte_reader_id = 0; bool is_eof = false; bool is_cancelled = false; diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 9bdc4094c67..a7aa4980209 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -19,9 +19,9 @@ namespace DB { -CTEOpStatus CTEReader::fetchNextBlock(Block & block) +CTEOpStatus CTEReader::fetchNextBlock(size_t source_id, Block & block) { - auto ret = this->cte->tryGetBlockAt(this->cte_reader_id, block); + auto ret = this->cte->tryGetBlockAt(this->cte_reader_id, source_id, block); switch (ret) { case CTEOpStatus::Eof: diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index be8a1f1c396..62e3357263b 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -25,24 +25,6 @@ namespace DB { -class CTEReaderNotifyFuture : public NotifyFuture -{ -public: - CTEReaderNotifyFuture(std::shared_ptr cte_, size_t cte_reader_id_) - : cte(cte_) - , cte_reader_id(cte_reader_id_) - {} - - void registerTask(TaskPtr && task) override - { - this->cte->checkBlockAvailableAndRegisterTask(std::move(task), this->cte_reader_id); - } - -private: - std::shared_ptr cte; - size_t cte_reader_id; -}; - class CTEReader { public: @@ -58,7 +40,6 @@ class CTEReader , cte(cte_manager_ ->getCTEBySource(query_id_and_cte_id_, partition_id, expected_sink_num_, expected_source_num_)) , cte_reader_id(this->cte->getCTEReaderID()) - , notifier(cte, this->cte_reader_id) {} ~CTEReader() @@ -67,7 +48,7 @@ class CTEReader this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id, this->partition_id); } - CTEOpStatus fetchNextBlock(Block & block); + CTEOpStatus fetchNextBlock(size_t source_id, Block & block); CTEOpStatus checkAvailableBlock(); void getResp(tipb::SelectResponse & resp) @@ -79,9 +60,8 @@ class CTEReader resp.CopyFrom(this->resp); } - void setNotifyFuture() { ::DB::setNotifyFuture(&(this->notifier)); } - std::shared_ptr getCTE() const { return this->cte; } + size_t getID() const { return this->cte_reader_id; } private: String query_id_and_cte_id; @@ -89,7 +69,6 @@ class CTEReader CTEManager * cte_manager; std::shared_ptr cte; size_t cte_reader_id; - CTEReaderNotifyFuture notifier; std::mutex mu; bool resp_fetched = false; diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 86b9c8a9f40..b11113d88a4 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -28,7 +28,7 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) return OperatorStatus::FINISHED; this->total_rows += block.rows(); - if (this->cte->pushBlock(block)) + if (this->cte->pushBlock(this->id, block)) return OperatorStatus::NEED_INPUT; const String & err = this->cte->getError(); if (err.empty()) diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index e1a8afb9474..2afcb2be600 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -18,16 +18,15 @@ #include #include -#include "Common/Stopwatch.h" - namespace DB { class CTESinkOp : public SinkOp { public: - CTESinkOp(PipelineExecutorContext & exec_context_, const String & req_id, std::shared_ptr cte_) + CTESinkOp(PipelineExecutorContext & exec_context_, const String & req_id, std::shared_ptr cte_, size_t id_) : SinkOp(exec_context_, req_id) , cte(cte_) + , id(id_) {} String getName() const override { return "CTESinkOp"; } @@ -40,5 +39,6 @@ class CTESinkOp : public SinkOp private: std::shared_ptr cte; size_t total_rows = 0; + size_t id; }; } // namespace DB diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index 4fcf2b17eb0..70bca0ee1ea 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -15,6 +15,7 @@ #include #include #include +#include namespace DB { @@ -25,7 +26,7 @@ void CTESourceOp::operateSuffixImpl() OperatorStatus CTESourceOp::readImpl(Block & block) { - auto ret = this->cte_reader->fetchNextBlock(block); + auto ret = this->cte_reader->fetchNextBlock(this->id, block); switch (ret) { case CTEOpStatus::Eof: @@ -36,7 +37,7 @@ OperatorStatus CTESourceOp::readImpl(Block & block) this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; case CTEOpStatus::BlockNotAvailable: - this->cte_reader->setNotifyFuture(); + DB::setNotifyFuture(&(this->notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::Cancelled: return OperatorStatus::CANCELLED; diff --git a/dbms/src/Operators/CTESourceOp.h b/dbms/src/Operators/CTESourceOp.h index 8959be5c06b..cdf22e1812d 100644 --- a/dbms/src/Operators/CTESourceOp.h +++ b/dbms/src/Operators/CTESourceOp.h @@ -26,6 +26,26 @@ namespace DB { +class CTESourceNotifyFuture : public NotifyFuture +{ +public: + CTESourceNotifyFuture(std::shared_ptr cte_, size_t cte_reader_id_, size_t source_id_) + : cte(cte_) + , cte_reader_id(cte_reader_id_) + , source_id(source_id_) + {} + + void registerTask(TaskPtr && task) override + { + this->cte->checkBlockAvailableAndRegisterTask(std::move(task), this->cte_reader_id, this->source_id); + } + +private: + std::shared_ptr cte; + size_t cte_reader_id; + size_t source_id; +}; + class CTESourceOp : public SourceOp { public: @@ -33,10 +53,13 @@ class CTESourceOp : public SourceOp PipelineExecutorContext & exec_context_, const String & req_id, std::shared_ptr cte_reader_, + size_t id_, const NamesAndTypes & schema) : SourceOp(exec_context_, req_id) , cte_reader(cte_reader_) , io_profile_info(IOProfileInfo::createForRemote(profile_info_ptr, 1)) + , id(id_) + , notifier(this->cte_reader->getCTE(), this->cte_reader->getID(), this->id) { setHeader(Block(getColumnWithTypeAndName(schema))); } @@ -54,5 +77,7 @@ class CTESourceOp : public SourceOp uint64_t total_rows{}; IOProfileInfoPtr io_profile_info; tipb::SelectResponse resp; + size_t id; + CTESourceNotifyFuture notifier; }; } // namespace DB From 5f9b624e56fecd047da3a023530d8616c0536525 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 20 Jun 2025 20:11:52 +0800 Subject: [PATCH 066/118] tweaking --- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 2 +- dbms/src/Operators/CTE.cpp | 18 +++++++++++------- dbms/src/Operators/CTE.h | 18 +++++++++++------- dbms/src/Operators/CTESourceOp.cpp | 2 +- 4 files changed, 24 insertions(+), 16 deletions(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index ce43100635d..c5698c6d5b3 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -64,7 +64,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( } size_t id = 0; - + group_builder.transform([&](auto & builder) { if (fine_grained_shuffle.enabled()) { diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index ce246984ba8..be8622fa91e 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -23,7 +23,10 @@ namespace DB { constexpr size_t PARTITION_NUM = 70; // TODO maybe need more tests to select a reasonable value -inline size_t getPartitionID(size_t id) { return id % PARTITION_NUM; } +inline size_t getPartitionID(size_t id) +{ + return id % PARTITION_NUM; +} CTE::CTE() { @@ -33,9 +36,10 @@ CTE::CTE() this->partitions.back().mu = std::make_unique(); this->partitions.back().read_mu = std::make_unique(); this->partitions.back().write_mu = std::make_unique(); + this->partitions.back().pipe_cv = std::make_unique(); } } - + CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & block) { auto partition_id = getPartitionID(source_id); @@ -68,14 +72,14 @@ bool CTE::pushBlock(size_t sink_id, const Block & block) std::lock_guard lock(*this->partitions[partition_id].mu); this->partitions[partition_id].memory_usages += block.bytes(); this->partitions[partition_id].blocks.push_back(block); - this->pipe_cv.notifyOne(); + this->partitions[partition_id].pipe_cv->notifyOne(); return true; } -void CTE::registerTask(TaskPtr && task, NotifyType type) +void CTE::registerTask(size_t partition_id, TaskPtr && task, NotifyType type) { task->setNotifyType(type); - pipe_cv.registerTask(std::move(task)); + this->partitions[partition_id].pipe_cv->registerTask(std::move(task)); } void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t source_id) @@ -91,10 +95,10 @@ void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_ if (status == CTEOpStatus::Ok) { - this->notifyTaskDirectly(std::move(task)); + this->notifyTaskDirectly(partition_id, std::move(task)); return; } - this->registerTask(std::move(task), NotifyType::WAIT_ON_CTE); + this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); } } // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 4c377b3c5d1..eb22a378f4f 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -60,6 +60,7 @@ struct CTEPartition Blocks blocks; std::unordered_map fetch_block_idxs; size_t memory_usages = 0; // TODO need a unified statistic + std::unique_ptr pipe_cv; }; class CTE @@ -97,8 +98,11 @@ class CTE void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t source_id); - void registerTask(TaskPtr && task, NotifyType type); - void notifyTaskDirectly(TaskPtr && task) { this->pipe_cv.notifyTaskDirectly(std::move(task)); } + void registerTask(size_t partition_id, TaskPtr && task, NotifyType type); + void notifyTaskDirectly(size_t partition_id, TaskPtr && task) + { + this->partitions[partition_id].pipe_cv->notifyTaskDirectly(std::move(task)); + } void addResp(const tipb::SelectResponse & resp) { @@ -122,7 +126,8 @@ class CTE if unlikely (this->is_cancelled) return CTEOpStatus::Cancelled; - if (this->partitions[partition_id].blocks.size() <= this->partitions[partition_id].fetch_block_idxs[cte_reader_id].idx) + if (this->partitions[partition_id].blocks.size() + <= this->partitions[partition_id].fetch_block_idxs[cte_reader_id].idx) return this->is_eof ? CTEOpStatus::Eof : CTEOpStatus::BlockNotAvailable; return CTEOpStatus::Ok; @@ -140,13 +145,12 @@ class CTE else this->is_cancelled = true; - // Just in case someone is in WAITING_FOR_NOTIFY status - this->pipe_cv.notifyAll(); + for (auto & partition : this->partitions) + partition.pipe_cv->notifyAll(); } std::vector partitions; - PipeConditionVariable pipe_cv; - + std::shared_mutex rw_lock; size_t next_cte_reader_id = 0; bool is_eof = false; diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index 70bca0ee1ea..f50680b7ac1 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include -#include namespace DB { From a3a96fd7bc2948a84f82ecccdcca6ddb785dec45 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Sun, 22 Jun 2025 19:02:29 +0800 Subject: [PATCH 067/118] tweaking --- dbms/src/Flash/Mpp/CTEManager.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index 53feba1fdca..b67fc0989a1 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -36,6 +36,7 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id, const St query_id_and_cte_id, partition_id); + iter_for_cte->second.sourceExit(); if (iter_for_cte->second.getTotalExitNum() == iter_for_cte->second.getExpectedTotalNum()) iter->second.erase(iter_for_cte); From 8107ac7a80b99dbcc29b14605ee5c6a325426ea7 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Sun, 22 Jun 2025 19:50:34 +0800 Subject: [PATCH 068/118] fix hang --- dbms/src/Flash/Mpp/CTEManager.cpp | 7 +++++-- dbms/src/Operators/CTE.cpp | 16 +++++++--------- dbms/src/Operators/CTE.h | 11 ++++------- 3 files changed, 16 insertions(+), 18 deletions(-) diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index b67fc0989a1..5ceae467e09 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -101,11 +101,14 @@ std::shared_ptr CTEManager::getCTEImpl( this->ctes[query_id_and_cte_id] = std::unordered_map{}; auto & cte_map = this->ctes[query_id_and_cte_id]; - auto [iter_for_cte, _] = cte_map.emplace( + auto [iter_for_cte, inserted] = cte_map.emplace( std::piecewise_construct, std::forward_as_tuple(partition_id), std::forward_as_tuple(std::make_shared(), expected_sink_num, expected_source_num)); - return iter_for_cte->second.getCTE(); + auto cte = iter_for_cte->second.getCTE(); + if (inserted) + cte->init(); + return cte; } } // namespace DB diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index be8622fa91e..6b427a5bedf 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -28,7 +28,7 @@ inline size_t getPartitionID(size_t id) return id % PARTITION_NUM; } -CTE::CTE() +void CTE::init() { for (size_t i = 0; i < PARTITION_NUM; i++) { @@ -87,18 +87,16 @@ void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_ auto partition_id = getPartitionID(source_id); CTEOpStatus status; - { - std::shared_lock rw_lock(this->rw_lock); - std::lock_guard lock(*this->partitions[partition_id].mu); - status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); - } + std::shared_lock rw_lock(this->rw_lock); + std::lock_guard lock(*this->partitions[partition_id].mu); + status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); - if (status == CTEOpStatus::Ok) + if (status == CTEOpStatus::BlockNotAvailable) { - this->notifyTaskDirectly(partition_id, std::move(task)); + this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); return; } - this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); + this->notifyTaskDirectly(partition_id, std::move(task)); } } // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index eb22a378f4f..68a2585f676 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -66,7 +66,7 @@ struct CTEPartition class CTE { public: - CTE(); + void init(); size_t getCTEReaderID() { @@ -81,8 +81,8 @@ class CTE CTEOpStatus tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & block); bool pushBlock(size_t sink_id, const Block & block); - void notifyEOF() { this->notifyImpl(true); } - void notifyCancel() { this->notifyImpl(false); } + void notifyEOF() { this->notifyImpl(true); } + void notifyCancel() { this->notifyImpl(false); } void notifyError(const String & err_msg) { @@ -133,12 +133,9 @@ class CTE return CTEOpStatus::Ok; } - template void notifyImpl(bool is_eof) { - std::unique_lock lock(this->rw_lock, std::defer_lock); - if constexpr (has_lock) - lock.lock(); + std::unique_lock lock(this->rw_lock); if likely (is_eof) this->is_eof = true; From a1c1d5d22e2646f10d0f7129bc37a597469469ff Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 24 Jun 2025 11:54:01 +0800 Subject: [PATCH 069/118] tweaking --- dbms/src/Flash/Coprocessor/DAGContext.h | 2 +- dbms/src/Flash/Executor/PipelineExecutorContext.cpp | 1 + dbms/src/Operators/CTE.cpp | 6 +----- dbms/src/Operators/CTE.h | 5 +---- 4 files changed, 4 insertions(+), 10 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 37de63de639..76fa708d629 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -36,7 +36,6 @@ #include #include #include -#include #include #include #include @@ -57,6 +56,7 @@ class CoprocessorReader; using CoprocessorReaderPtr = std::shared_ptr; class AutoSpillTrigger; +class CTE; struct JoinProfileInfo { diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.cpp b/dbms/src/Flash/Executor/PipelineExecutorContext.cpp index 8b7352ca3a0..6aa8c3a53dd 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 6b427a5bedf..5f30372b323 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -21,7 +21,7 @@ namespace DB { -constexpr size_t PARTITION_NUM = 70; // TODO maybe need more tests to select a reasonable value +constexpr size_t PARTITION_NUM = 30; inline size_t getPartitionID(size_t id) { @@ -34,8 +34,6 @@ void CTE::init() { this->partitions.push_back(CTEPartition()); this->partitions.back().mu = std::make_unique(); - this->partitions.back().read_mu = std::make_unique(); - this->partitions.back().write_mu = std::make_unique(); this->partitions.back().pipe_cv = std::make_unique(); } } @@ -45,7 +43,6 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & b auto partition_id = getPartitionID(source_id); std::shared_lock rw_lock(this->rw_lock); - std::lock_guard read_lock(*this->partitions[partition_id].read_mu); std::lock_guard lock(*this->partitions[partition_id].mu); auto status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); if (status != CTEOpStatus::Ok) @@ -68,7 +65,6 @@ bool CTE::pushBlock(size_t sink_id, const Block & block) return true; auto partition_id = getPartitionID(sink_id); - std::lock_guard write_lock(*this->partitions[partition_id].write_mu); std::lock_guard lock(*this->partitions[partition_id].mu); this->partitions[partition_id].memory_usages += block.bytes(); this->partitions[partition_id].blocks.push_back(block); diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 68a2585f676..a95ad2507f5 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -16,9 +16,8 @@ #include #include -#include -#include #include +#include #include #include @@ -55,8 +54,6 @@ struct IdxWithPadding struct CTEPartition { std::unique_ptr mu; - std::unique_ptr read_mu; - std::unique_ptr write_mu; Blocks blocks; std::unordered_map fetch_block_idxs; size_t memory_usages = 0; // TODO need a unified statistic From 55bccf562d9c96bfb285db66a859b5f23d350302 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 24 Jun 2025 14:36:58 +0800 Subject: [PATCH 070/118] remove fine grained --- .../Executor/PipelineExecutorContext.cpp | 2 +- dbms/src/Flash/Mpp/CTEManager.cpp | 70 +++++-------------- dbms/src/Flash/Mpp/CTEManager.h | 14 ++-- .../Schedule/Tasks/PipeConditionVariable.h | 1 + .../Flash/Planner/Plans/PhysicalCTESink.cpp | 28 +++----- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 39 +++-------- dbms/src/Operators/CTE.cpp | 23 +----- dbms/src/Operators/CTE.h | 16 ++++- dbms/src/Operators/CTEReader.h | 8 +-- 9 files changed, 66 insertions(+), 135 deletions(-) diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.cpp b/dbms/src/Flash/Executor/PipelineExecutorContext.cpp index 6aa8c3a53dd..aedcd1f159e 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.cpp @@ -20,8 +20,8 @@ #include #include #include -#include #include +#include #include diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index 5ceae467e09..46dc9744973 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -21,64 +21,34 @@ namespace DB { -void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id, const String & partition_id) +void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id) { std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if unlikely (iter == this->ctes.end()) - // Maybe the task is cancelled and all ctes have been released + // Maybe the task is cancelled and the cte has been released return; - auto iter_for_cte = iter->second.find(partition_id); - RUNTIME_CHECK_MSG( - iter_for_cte != iter->second.end(), - "Can't find cte: {}, partition: {}", - query_id_and_cte_id, - partition_id); - - iter_for_cte->second.sourceExit(); - if (iter_for_cte->second.getTotalExitNum() == iter_for_cte->second.getExpectedTotalNum()) - iter->second.erase(iter_for_cte); - - if (iter->second.empty()) + iter->second.sourceExit(); + if (iter->second.getTotalExitNum() == iter->second.getExpectedTotalNum()) this->ctes.erase(iter); } -// TODO refine codes here, do not directly use map void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) { std::unique_lock lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if unlikely (iter == this->ctes.end()) - // Maybe the task is cancelled and all ctes have been released + // Maybe the task is cancelled and the cte has been released return; - auto iter_for_cte = iter->second.begin(); - auto iter_for_cte_end = iter->second.end(); - std::vector ctes_need_erase; - while (iter_for_cte != iter_for_cte_end) - { - CTEWithCounter & cte_with_counter = iter_for_cte->second; - cte_with_counter.getCTE()->addResp(resp); - cte_with_counter.sinkExit(); - if (cte_with_counter.getSinkExitNum() == cte_with_counter.getExpectedSinkNum()) - cte_with_counter.getCTE()->notifyEOF(); - if (cte_with_counter.getTotalExitNum() == cte_with_counter.getExpectedTotalNum()) - ctes_need_erase.push_back(iter_for_cte->first); - iter_for_cte++; - } - - if (ctes_need_erase.size() == iter->second.size()) - { + CTEWithCounter & cte_with_counter = iter->second; + cte_with_counter.getCTE()->addResp(resp); + cte_with_counter.sinkExit(); + if (cte_with_counter.getSinkExitNum() == cte_with_counter.getExpectedSinkNum()) + cte_with_counter.getCTE()->notifyEOF(); + if (cte_with_counter.getTotalExitNum() == cte_with_counter.getExpectedTotalNum()) this->ctes.erase(iter); - return; - } - - for (const auto & key : ctes_need_erase) - { - auto iter_for_cte = iter->second.find(key); - iter->second.erase(iter_for_cte); - } } void CTEManager::releaseCTEs(const String & query_id_and_cte_id) @@ -91,24 +61,16 @@ void CTEManager::releaseCTEs(const String & query_id_and_cte_id) std::shared_ptr CTEManager::getCTEImpl( const String & query_id_and_cte_id, - const String & partition_id, + Int32 concurrency, Int32 expected_sink_num, Int32 expected_source_num) { std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if (iter == this->ctes.end()) - this->ctes[query_id_and_cte_id] = std::unordered_map{}; - - auto & cte_map = this->ctes[query_id_and_cte_id]; - auto [iter_for_cte, inserted] = cte_map.emplace( - std::piecewise_construct, - std::forward_as_tuple(partition_id), - std::forward_as_tuple(std::make_shared(), expected_sink_num, expected_source_num)); - - auto cte = iter_for_cte->second.getCTE(); - if (inserted) - cte->init(); - return cte; + this->ctes.insert(std::make_pair( + query_id_and_cte_id, + CTEWithCounter(std::make_shared(concurrency), expected_sink_num, expected_source_num))); + return this->ctes.find(query_id_and_cte_id)->second.getCTE(); } } // namespace DB diff --git a/dbms/src/Flash/Mpp/CTEManager.h b/dbms/src/Flash/Mpp/CTEManager.h index 60189390c9c..0b5f304c31c 100644 --- a/dbms/src/Flash/Mpp/CTEManager.h +++ b/dbms/src/Flash/Mpp/CTEManager.h @@ -58,32 +58,32 @@ class CTEManager public: std::shared_ptr getCTEBySink( const String & query_id_and_cte_id, - const String & partition_id, + Int32 concurrency, Int32 expected_sink_num, Int32 expected_source_num) { - return this->getCTEImpl(query_id_and_cte_id, partition_id, expected_sink_num, expected_source_num); + return this->getCTEImpl(query_id_and_cte_id, concurrency, expected_sink_num, expected_source_num); } std::shared_ptr getCTEBySource( const String & query_id_and_cte_id, - const String & partition_id, + Int32 concurrency, Int32 expected_sink_num, Int32 expected_source_num) { - return this->getCTEImpl(query_id_and_cte_id, partition_id, expected_sink_num, expected_source_num); + return this->getCTEImpl(query_id_and_cte_id, concurrency, expected_sink_num, expected_source_num); } - void releaseCTEBySource(const String & query_id_and_cte_id, const String & partition_id); + void releaseCTEBySource(const String & query_id_and_cte_id); void releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id); void releaseCTEs(const String & query_id_and_cte_id); private: std::shared_ptr getCTEImpl( const String & query_id_and_cte_id, - const String & partition_id, + Int32 concurrency, Int32 expected_sink_num, Int32 expected_source_num); std::mutex mu; - std::unordered_map> ctes; + std::unordered_map ctes; }; } // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index e5bbb90ac45..46476532f6c 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -59,6 +59,7 @@ class PipeConditionVariable break; case NotifyType::WAIT_ON_CTE: GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte).Increment(change); + break; case NotifyType::WAIT_ON_NOTHING: throw Exception("task notify type should be set before register or notify"); break; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index c5698c6d5b3..a7931089546 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -50,32 +50,22 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( PipelineExecutorContext & exec_context, PipelineExecGroupBuilder & group_builder, Context & context, - size_t /*concurrency*/) + size_t concurrency) { - size_t partition_id = 0; String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); exec_context.setQueryIDAndCTEID(query_id_and_cte_id); - std::shared_ptr cte; - if (!fine_grained_shuffle.enabled()) - { - cte = context.getCTEManager() - ->getCTEBySink(query_id_and_cte_id, "", this->expected_sink_num, this->expected_source_num); - } + if (fine_grained_shuffle.enabled()) + concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); - size_t id = 0; + std::shared_ptr cte = context.getCTEManager()->getCTEBySink( + query_id_and_cte_id, + concurrency, + this->expected_sink_num, + this->expected_source_num); + size_t id = 0; group_builder.transform([&](auto & builder) { - if (fine_grained_shuffle.enabled()) - { - cte = context.getCTEManager()->getCTEBySink( - query_id_and_cte_id, - std::to_string(partition_id), - this->expected_sink_num, - this->expected_source_num); - partition_id++; - } - builder.setSinkOp(std::make_unique(exec_context, log->identifier(), cte, id)); id++; }); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 2c24bb9402a..db01550f8b8 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -66,37 +66,20 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( exec_context.setQueryIDAndCTEID(query_id_and_cte_id); exec_context.setHasCTESource(); - if (fine_grained_shuffle.enabled()) - { - for (size_t partition_id = 0; partition_id < concurrency; ++partition_id) - { - auto cte_reader = std::make_shared( - query_id_and_cte_id, - std::to_string(partition_id), - context.getCTEManager(), - this->expected_sink_num, - this->expected_source_num); - exec_context.addCTE(cte_reader->getCTE()); - group_builder.addConcurrency( - std::make_unique(exec_context, log->identifier(), cte_reader, partition_id, schema)); - } - } - else + auto cte_reader = std::make_shared( + query_id_and_cte_id, + concurrency, + context.getCTEManager(), + this->expected_sink_num, + this->expected_source_num); + exec_context.addCTE(cte_reader->getCTE()); + for (size_t i = 0; i < concurrency; ++i) { - auto cte_reader = std::make_shared( - query_id_and_cte_id, - "", - context.getCTEManager(), - this->expected_sink_num, - this->expected_source_num); - exec_context.addCTE(cte_reader->getCTE()); - for (size_t i = 0; i < concurrency; ++i) - { - group_builder.addConcurrency( - std::make_unique(exec_context, log->identifier(), cte_reader, i, schema)); - } + group_builder.addConcurrency( + std::make_unique(exec_context, log->identifier(), cte_reader, i, schema)); } + context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 5f30372b323..4488fd22fbe 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -21,26 +21,9 @@ namespace DB { -constexpr size_t PARTITION_NUM = 30; - -inline size_t getPartitionID(size_t id) -{ - return id % PARTITION_NUM; -} - -void CTE::init() -{ - for (size_t i = 0; i < PARTITION_NUM; i++) - { - this->partitions.push_back(CTEPartition()); - this->partitions.back().mu = std::make_unique(); - this->partitions.back().pipe_cv = std::make_unique(); - } -} - CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & block) { - auto partition_id = getPartitionID(source_id); + auto partition_id = this->getPartitionID(source_id); std::shared_lock rw_lock(this->rw_lock); std::lock_guard lock(*this->partitions[partition_id].mu); @@ -64,7 +47,7 @@ bool CTE::pushBlock(size_t sink_id, const Block & block) if unlikely (block.rows() == 0) return true; - auto partition_id = getPartitionID(sink_id); + auto partition_id = this->getPartitionID(sink_id); std::lock_guard lock(*this->partitions[partition_id].mu); this->partitions[partition_id].memory_usages += block.bytes(); this->partitions[partition_id].blocks.push_back(block); @@ -80,7 +63,7 @@ void CTE::registerTask(size_t partition_id, TaskPtr && task, NotifyType type) void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t source_id) { - auto partition_id = getPartitionID(source_id); + auto partition_id = this->getPartitionID(source_id); CTEOpStatus status; std::shared_lock rw_lock(this->rw_lock); diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index a95ad2507f5..d919f39859c 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -16,8 +16,8 @@ #include #include -#include #include +#include #include #include @@ -63,6 +63,17 @@ struct CTEPartition class CTE { public: + explicit CTE(size_t partition_num_) + : partition_num(partition_num_) + { + for (size_t i = 0; i < this->partition_num; i++) + { + this->partitions.push_back(CTEPartition()); + this->partitions.back().mu = std::make_unique(); + this->partitions.back().pipe_cv = std::make_unique(); + } + } + void init(); size_t getCTEReaderID() @@ -118,6 +129,8 @@ class CTE } private: + size_t getPartitionID(size_t id) const { return id % this->partition_num; } + CTEOpStatus checkBlockAvailableNoLock(size_t cte_reader_id, size_t partition_id) { if unlikely (this->is_cancelled) @@ -143,6 +156,7 @@ class CTE partition.pipe_cv->notifyAll(); } + size_t partition_num; std::vector partitions; std::shared_mutex rw_lock; diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 62e3357263b..9c57b179cdb 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -30,22 +30,21 @@ class CTEReader public: CTEReader( const String & query_id_and_cte_id_, - const String & partition_id_, + size_t partition_num, CTEManager * cte_manager_, Int32 expected_sink_num_, Int32 expected_source_num_) : query_id_and_cte_id(query_id_and_cte_id_) - , partition_id(partition_id_) , cte_manager(cte_manager_) , cte(cte_manager_ - ->getCTEBySource(query_id_and_cte_id_, partition_id, expected_sink_num_, expected_source_num_)) + ->getCTEBySource(query_id_and_cte_id_, partition_num, expected_sink_num_, expected_source_num_)) , cte_reader_id(this->cte->getCTEReaderID()) {} ~CTEReader() { this->cte.reset(); - this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id, this->partition_id); + this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id); } CTEOpStatus fetchNextBlock(size_t source_id, Block & block); @@ -65,7 +64,6 @@ class CTEReader private: String query_id_and_cte_id; - String partition_id; CTEManager * cte_manager; std::shared_ptr cte; size_t cte_reader_id; From 2f59cc239c9a4e707b4735b298db9bfdf7145d33 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 24 Jun 2025 14:47:24 +0800 Subject: [PATCH 071/118] tweaking --- dbms/src/Operators/CTE.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index d919f39859c..1daa02da403 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -74,8 +74,6 @@ class CTE } } - void init(); - size_t getCTEReaderID() { std::unique_lock lock(this->rw_lock); From 17ac8bde9873429f1b2d0b5515c07b25bb7402db Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 25 Jun 2025 13:33:03 +0800 Subject: [PATCH 072/118] fix ci --- .../Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp index e8f102996f5..b07831bb123 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp @@ -455,7 +455,7 @@ TEST_F(TestResourceControlQueue, RunOutOfRU) TaskScheduler task_scheduler(config); PipelineExecutorContext - exec_context("mock-query-id", "mock-req-id", mem_tracker, nullptr, nullptr, nullptr, NullspaceID, rg_name); + exec_context("mock-query-id", "mock-query-id-for-cte", "mock-req-id", mem_tracker, nullptr, nullptr, nullptr, NullspaceID, rg_name); auto task = std::make_unique(exec_context); // This task should use 5*100ms cpu_time. From bd86c7873121eec882f454838a01b760eec9aae7 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 25 Jun 2025 14:53:46 +0800 Subject: [PATCH 073/118] address comments --- .../Executor/PipelineExecutorContext.cpp | 2 +- dbms/src/Flash/Mpp/CTEManager.cpp | 2 +- dbms/src/Flash/Mpp/CTEManager.h | 2 +- dbms/src/Flash/Mpp/MPPTask.cpp | 24 ++++++++------- dbms/src/Flash/Mpp/MPPTask.h | 5 ++-- .../tests/gtest_resource_control_queue.cpp | 12 ++++++-- dbms/src/Operators/CTE.cpp | 4 +-- dbms/src/Operators/CTE.h | 30 ++++++++----------- dbms/src/Operators/CTEReader.cpp | 9 +++--- dbms/src/Operators/CTEReader.h | 1 - dbms/src/Operators/CTESinkOp.cpp | 5 +--- dbms/src/Operators/CTESinkOp.h | 1 - dbms/src/Operators/CTESourceOp.cpp | 8 ++--- 13 files changed, 51 insertions(+), 54 deletions(-) diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.cpp b/dbms/src/Flash/Executor/PipelineExecutorContext.cpp index aedcd1f159e..f2a3609d6c2 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.cpp @@ -189,7 +189,7 @@ void PipelineExecutorContext::cancel() auto ctes = dag_context->getCTEs(); if (!ctes.empty()) - ctes[0]->notifyError(getTrimmedErrMsg()); + ctes[0]->notifyCancel(getTrimmedErrMsg()); if (auto mpp_receiver_set = dag_context->getMPPReceiverSet(); mpp_receiver_set) mpp_receiver_set->cancel(); diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index 46dc9744973..edb177467ed 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -51,7 +51,7 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin this->ctes.erase(iter); } -void CTEManager::releaseCTEs(const String & query_id_and_cte_id) +void CTEManager::releaseCTE(const String & query_id_and_cte_id) { std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); diff --git a/dbms/src/Flash/Mpp/CTEManager.h b/dbms/src/Flash/Mpp/CTEManager.h index 0b5f304c31c..3a13c2eb08d 100644 --- a/dbms/src/Flash/Mpp/CTEManager.h +++ b/dbms/src/Flash/Mpp/CTEManager.h @@ -74,7 +74,7 @@ class CTEManager } void releaseCTEBySource(const String & query_id_and_cte_id); void releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id); - void releaseCTEs(const String & query_id_and_cte_id); + void releaseCTE(const String & query_id_and_cte_id); private: std::shared_ptr getCTEImpl( diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 44884306086..72e0c5d5f67 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -153,6 +153,7 @@ MPPTask::~MPPTask() if (query_memory_tracker != nullptr && current_memory_tracker != query_memory_tracker) current_memory_tracker = query_memory_tracker; abortTunnels("", true); + abortCTE(""); LOG_INFO(log, "finish MPPTask: {}, total run time is {} ms", id.toString(), total_run_time_ms); } @@ -190,9 +191,9 @@ void MPPTask::abortQueryExecutor() } } -void MPPTask::abortCTE() +void MPPTask::abortCTE(const String & message) { - if (this->dag_context->hasCTESource()) + if (this->has_cte_sink.load()) { auto ctes = this->dag_context->getCTEs(); @@ -200,14 +201,14 @@ void MPPTask::abortCTE() // We'd better to manually do notification in case of missing signal from cte sink // or the CTESource will hang for (auto & cte : ctes) - cte->notifyCancel(); - this->context->getCTEManager()->releaseCTEs(this->dag_context->getQueryIDAndCTEID()); + cte->notifyCancel(message); + this->context->getCTEManager()->releaseCTE(this->dag_context->getQueryIDAndCTEID()); } } void MPPTask::finishWrite() { - if (this->has_cte_sink) + if (this->has_cte_sink.load()) { tipb::SelectResponse resp; if (dag_context->collect_execution_summaries) @@ -238,11 +239,11 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) { if unlikely (!dag_context->dag_request.rootExecutor().has_exchange_sender()) { - if unlikely (!dag_context->dag_request.rootExecutor().has_cte_sink()) - throw Exception("Task has either exchange sender or cte sink"); - + RUNTIME_CHECK_MSG( + dag_context->dag_request.rootExecutor().has_cte_sink(), + "Task should has either exchange sender or cte sink"); // There is no need to register tunnel for cte sink - this->has_cte_sink = true; + this->has_cte_sink.store(true); return; } auto tunnel_set_local = std::make_shared(log->identifier()); @@ -669,7 +670,7 @@ void MPPTask::runImpl() err_msg = err_msg.empty() ? catch_err_msg : fmt::format("{}, {}", err_msg, catch_err_msg); } - if (this->has_cte_sink && !this->notify_cte_finish) + if (this->has_cte_sink.load() && !this->notify_cte_finish) { tipb::SelectResponse resp; this->context->getCTEManager()->releaseCTEBySink(resp, this->dag_context->getQueryIDAndCTEID()); @@ -817,6 +818,7 @@ void MPPTask::abort(const String & message, AbortType abort_type) /// if the task is in initializing state, mpp task can return error to TiDB directly, /// so just close all tunnels here abortTunnels("", false); + abortCTE(message); LOG_WARNING(log, "Finish abort task from uninitialized"); break; } @@ -829,7 +831,7 @@ void MPPTask::abort(const String & message, AbortType abort_type) abortTunnels(message, false); abortReceivers(); abortQueryExecutor(); - abortCTE(); + abortCTE(message); scheduleThisTask(ScheduleState::FAILED); /// runImpl is running, leave remaining work to runImpl LOG_WARNING(log, "Finish abort task from running"); diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index 54dd11af393..512f68f50d5 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -124,7 +124,7 @@ class MPPTask void abortTunnels(const String & message, bool wait_sender_finish); void abortReceivers(); void abortQueryExecutor(); - void abortCTE(); + void abortCTE(const String & message); void finishWrite(); @@ -200,8 +200,7 @@ class MPPTask MPPReceiverSetPtr receiver_set; - bool has_cte_sink = false; - bool has_cte_source = false; + std::atomic has_cte_sink = false; bool notify_cte_finish = false; int new_thread_count_of_mpp_receiver = 0; diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp index b07831bb123..a26212fde06 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp @@ -454,8 +454,16 @@ TEST_F(TestResourceControlQueue, RunOutOfRU) TaskSchedulerConfig config{thread_num, thread_num}; TaskScheduler task_scheduler(config); - PipelineExecutorContext - exec_context("mock-query-id", "mock-query-id-for-cte", "mock-req-id", mem_tracker, nullptr, nullptr, nullptr, NullspaceID, rg_name); + PipelineExecutorContext exec_context( + "mock-query-id", + "mock-query-id-for-cte", + "mock-req-id", + mem_tracker, + nullptr, + nullptr, + nullptr, + NullspaceID, + rg_name); auto task = std::make_unique(exec_context); // This task should use 5*100ms cpu_time. diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 4488fd22fbe..e8517101ebe 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -28,7 +28,7 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & b std::shared_lock rw_lock(this->rw_lock); std::lock_guard lock(*this->partitions[partition_id].mu); auto status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); - if (status != CTEOpStatus::Ok) + if (status != CTEOpStatus::OK) return status; auto idx = this->partitions[partition_id].fetch_block_idxs[cte_reader_id].idx++; @@ -70,7 +70,7 @@ void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_ std::lock_guard lock(*this->partitions[partition_id].mu); status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); - if (status == CTEOpStatus::BlockNotAvailable) + if (status == CTEOpStatus::BLOCK_NOT_AVAILABLE) { this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); return; diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 1daa02da403..d67f266c34d 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -31,11 +31,10 @@ namespace DB { enum class CTEOpStatus { - Ok, - BlockNotAvailable, - Eof, - Cancelled, - Error + OK, + BLOCK_NOT_AVAILABLE, + END_OF_FILE, + CANCELLED }; struct IdxWithPadding @@ -87,14 +86,8 @@ class CTE CTEOpStatus tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & block); bool pushBlock(size_t sink_id, const Block & block); - void notifyEOF() { this->notifyImpl(true); } - void notifyCancel() { this->notifyImpl(false); } - - void notifyError(const String & err_msg) - { - std::unique_lock lock(this->rw_lock); - this->err_msg = err_msg; - } + void notifyEOF() { this->notifyImpl(true, ""); } + void notifyCancel(const String & msg) { this->notifyImpl(false, msg); } String getError() { @@ -132,23 +125,26 @@ class CTE CTEOpStatus checkBlockAvailableNoLock(size_t cte_reader_id, size_t partition_id) { if unlikely (this->is_cancelled) - return CTEOpStatus::Cancelled; + return CTEOpStatus::CANCELLED; if (this->partitions[partition_id].blocks.size() <= this->partitions[partition_id].fetch_block_idxs[cte_reader_id].idx) - return this->is_eof ? CTEOpStatus::Eof : CTEOpStatus::BlockNotAvailable; + return this->is_eof ? CTEOpStatus::END_OF_FILE : CTEOpStatus::BLOCK_NOT_AVAILABLE; - return CTEOpStatus::Ok; + return CTEOpStatus::OK; } - void notifyImpl(bool is_eof) + void notifyImpl(bool is_eof, const String & msg) { std::unique_lock lock(this->rw_lock); if likely (is_eof) this->is_eof = true; else + { this->is_cancelled = true; + this->err_msg = msg; + } for (auto & partition : this->partitions) partition.pipe_cv->notifyAll(); diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index a7aa4980209..d39f1f3c2a4 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -24,17 +24,16 @@ CTEOpStatus CTEReader::fetchNextBlock(size_t source_id, Block & block) auto ret = this->cte->tryGetBlockAt(this->cte_reader_id, source_id, block); switch (ret) { - case CTEOpStatus::Eof: + case CTEOpStatus::END_OF_FILE: { std::lock_guard lock(this->mu); if (this->resp.execution_summaries_size() == 0) this->cte->tryToGetResp(this->resp); } - case CTEOpStatus::BlockNotAvailable: - case CTEOpStatus::Cancelled: - case CTEOpStatus::Ok: + case CTEOpStatus::BLOCK_NOT_AVAILABLE: + case CTEOpStatus::OK: return ret; - case DB::CTEOpStatus::Error: + case CTEOpStatus::CANCELLED: throw Exception(this->cte->getError()); } throw Exception("Should not reach here"); diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 9c57b179cdb..4f6ff82da45 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -48,7 +48,6 @@ class CTEReader } CTEOpStatus fetchNextBlock(size_t source_id, Block & block); - CTEOpStatus checkAvailableBlock(); void getResp(tipb::SelectResponse & resp) { diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index b11113d88a4..159483bca60 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -30,9 +30,6 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) this->total_rows += block.rows(); if (this->cte->pushBlock(this->id, block)) return OperatorStatus::NEED_INPUT; - const String & err = this->cte->getError(); - if (err.empty()) - return OperatorStatus::CANCELLED; - throw Exception(err); + return OperatorStatus::CANCELLED; } } // namespace DB diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 2afcb2be600..01bb8fcd623 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -30,7 +30,6 @@ class CTESinkOp : public SinkOp {} String getName() const override { return "CTESinkOp"; } - bool canHandleSelectiveBlock() const override { return true; } protected: void operateSuffixImpl() override; diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index f50680b7ac1..9883b1980ab 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -29,18 +29,16 @@ OperatorStatus CTESourceOp::readImpl(Block & block) auto ret = this->cte_reader->fetchNextBlock(this->id, block); switch (ret) { - case CTEOpStatus::Eof: + case CTEOpStatus::END_OF_FILE: this->cte_reader->getResp(this->resp); if (this->resp.execution_summaries_size() != 0) this->io_profile_info->remote_execution_summary.add(this->resp); - case CTEOpStatus::Ok: + case CTEOpStatus::OK: this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; - case CTEOpStatus::BlockNotAvailable: + case CTEOpStatus::BLOCK_NOT_AVAILABLE: DB::setNotifyFuture(&(this->notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; - case CTEOpStatus::Cancelled: - return OperatorStatus::CANCELLED; default: throw Exception("Should not reach here"); } From 65548d34c105160165355a7f3a996bf67ea0b65a Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 27 Jun 2025 14:03:19 +0800 Subject: [PATCH 074/118] save --- dbms/src/Core/CTESpill.cpp | 92 -------------- dbms/src/Core/CTESpill.h | 58 --------- dbms/src/Core/Spiller.h | 3 +- dbms/src/Interpreters/CTESpillContext.cpp | 36 ++++++ dbms/src/Interpreters/CTESpillContext.h | 54 +++++++++ dbms/src/Operators/CTE.cpp | 139 ++++++++-------------- dbms/src/Operators/CTE.h | 65 +--------- dbms/src/Operators/CTEPartition.cpp | 82 +++++++++++++ dbms/src/Operators/CTEPartition.h | 94 +++++++++++++++ 9 files changed, 325 insertions(+), 298 deletions(-) delete mode 100644 dbms/src/Core/CTESpill.cpp delete mode 100644 dbms/src/Core/CTESpill.h create mode 100644 dbms/src/Interpreters/CTESpillContext.cpp create mode 100644 dbms/src/Interpreters/CTESpillContext.h create mode 100644 dbms/src/Operators/CTEPartition.cpp create mode 100644 dbms/src/Operators/CTEPartition.h diff --git a/dbms/src/Core/CTESpill.cpp b/dbms/src/Core/CTESpill.cpp deleted file mode 100644 index 686a3030b0c..00000000000 --- a/dbms/src/Core/CTESpill.cpp +++ /dev/null @@ -1,92 +0,0 @@ -// Copyright 2025 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include - -#include -#include -#include - -namespace DB -{ -void CTESpill::writeBlocks(const Blocks & blocks) -{ - std::unique_lock lock(this->rw_lock); - - for (const auto & block : blocks) - { - if unlikely (this->spilled_files.back().isFull() || this->spilled_files.empty()) - { - // TODO create new spilled_files and write_read_files - } - - Int64 prev_block_offset; - Int64 current_block_file_idx = this->spilled_files.size() - 1; - if unlikely (this->block_offsets.empty() || this->block_offsets.back().first != current_block_file_idx) - prev_block_offset = 0; - else - prev_block_offset = this->block_offsets.back().second; - - const auto block_size = this->write_streams.back().writeAndReturnBlockSize(block); - this->block_offsets.push_back((std::make_pair(this->spilled_files.size() - 1, prev_block_offset + block_size))); - } -} - -Block CTESpill::readBlockAt(Int64 idx) -{ - std::shared_lock lock(this->rw_lock); - if unlikely (idx >= static_cast(this->block_offsets.size())) - throw Exception(fmt::format( - "Requested block idx({}) is larger than total block number({})", - idx, - this->block_offsets.size())); - - auto block_location = this->block_offsets[idx]; - Int64 stream_idx = block_location.first; - Int64 block_offset = block_location.second; - Int64 block_size = this->getBlockSizeNoLock(idx); - - // TODO why we resize buf? Maybe we can delete this code. - if (this->buf.size() < static_cast(block_size)) - this->buf.resize(block_size); - - this->read_streams[stream_idx].seek(block_offset); - return this->read_streams[stream_idx].read(); -} - -Int64 CTESpill::blockNum() -{ - std::shared_lock lock(this->rw_lock); - return this->block_offsets.size(); -} - -Int64 CTESpill::getBlockSizeNoLock(Int64 idx) const -{ - if unlikely (idx >= static_cast(this->block_offsets.size())) - throw Exception(fmt::format( - "Requested block idx({}) is larger than total block number({})", - idx, - this->block_offsets.size())); - - if unlikely (idx == 0) - return this->block_offsets[0].second; - - Int64 prev_block_file_idx = this->block_offsets[idx - 1].first; - if unlikely (prev_block_file_idx != this->block_offsets[idx].first) - return this->block_offsets[idx].second; - - return this->block_offsets[idx].second - this->block_offsets[idx - 1].second; -} -} // namespace DB diff --git a/dbms/src/Core/CTESpill.h b/dbms/src/Core/CTESpill.h deleted file mode 100644 index 96d25889977..00000000000 --- a/dbms/src/Core/CTESpill.h +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2025 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -namespace DB -{ -class CTESpill -{ -public: - // TODO maybe need refine - explicit CTESpill() = default; - - // TODO maybe we need an initialization function as spill may not be triggered and we can initialize it until spill is triggered - - // TODO all function need lock as CTESpill may be concurrently accessed - void writeBlocks(const Blocks & blocks); // TODO implement, return value need other types - Block readBlockAt(Int64 idx); - Int64 blockNum(); - -private: - Int64 getBlockSizeNoLock(Int64 idx) const; - - std::shared_mutex rw_lock; - std::vector spilled_files; - // TODO maybe we need to use compressed stream - // TODO maybe each one stream need one lock - std::vector read_streams; - std::vector write_streams; - std::vector> block_offsets; - FileProviderPtr file_provider; - // SpillConfig config; // TODO initialize it - - std::vector buf; -}; -} // namespace DB diff --git a/dbms/src/Core/Spiller.h b/dbms/src/Core/Spiller.h index 435bcdc396a..7eb43354b61 100644 --- a/dbms/src/Core/Spiller.h +++ b/dbms/src/Core/Spiller.h @@ -161,10 +161,11 @@ class Spiller /// file is read, otherwise, the spilled file will be released when destruct the spiller. Currently, all the spilled /// file can be released on restore since it is only read once, but in the future if SharedScan(shared cte) need spill, /// the data may be restored multiple times and release_spilled_file_on_restore need to be set to false. - const bool release_spilled_file_on_restore; + bool release_spilled_file_on_restore; bool enable_append_write = false; }; using SpillerPtr = std::unique_ptr; +using SpillerSharedPtr = std::shared_ptr; } // namespace DB diff --git a/dbms/src/Interpreters/CTESpillContext.cpp b/dbms/src/Interpreters/CTESpillContext.cpp new file mode 100644 index 00000000000..e2ac9f1d9b2 --- /dev/null +++ b/dbms/src/Interpreters/CTESpillContext.cpp @@ -0,0 +1,36 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +#include + +namespace DB +{ +SpillerSharedPtr CTESpillContext::getSpillAt(size_t idx) +{ + std::lock_guard lock(this->mu); + auto spiller_num = this->spillers.size(); + + // The spiller whose idx is lower that the parameter idx must have been created + RUNTIME_CHECK_MSG(idx <= spiller_num, "idx: {}, spiller_num: {}", idx, spiller_num); + + if (idx < spiller_num) + return this->spillers[idx]; + + + return this->spillers[idx]; +} +} // namespace DB diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h new file mode 100644 index 00000000000..ea7f978fb6a --- /dev/null +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -0,0 +1,54 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ +class CTESpillContext final : public OperatorSpillContext +{ +public: + CTESpillContext( + const SpillConfig & spill_config_, + const Block & spill_block_schema_, + UInt64 operator_spill_threshold_, + const LoggerPtr & log_) + : OperatorSpillContext(operator_spill_threshold_, "cte", log_) + , spill_config(spill_config_) + , spill_block_schema(spill_block_schema_) + {} + + ~CTESpillContext() override = default; + + Int64 getTotalRevocableMemoryImpl() override { throw Exception(""); } // TODO implement + bool supportFurtherSpill() const override { throw Exception(""); } // TODO implement + bool supportAutoTriggerSpill() const override { throw Exception(""); } // TODO implement + Int64 triggerSpillImpl(Int64) override { throw Exception(""); } // TODO implement + + SpillerSharedPtr getSpillAt(size_t idx); + +private: + std::mutex mu; + SpillConfig spill_config; + Block spill_block_schema; + + std::vector spillers; +}; +} // namespace DB diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index d699134dd85..1eab58bc4e8 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -25,62 +26,35 @@ namespace DB { CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & block) { + auto partition_id = this->getPartitionID(source_id); + { - std::shared_lock status_lock(this->aux_rw_lock); - if (this->cte_status != CTEStatus::NORMAL) - return CTEOpStatus::IO_OUT; + std::shared_lock rw_lock(this->rw_lock); + if unlikely (this->is_cancelled) + return CTEOpStatus::CANCELLED; } - std::shared_lock rw_lock(this->rw_lock); + std::lock_guard status_lock(this->partitions[partition_id].aux_lock); + if (this->partitions[partition_id].status != CTEPartitionStatus::NORMAL) + return CTEOpStatus::IO_OUT; - if unlikely (this->is_cancelled) - return CTEOpStatus::CANCELLED; - - auto partition_id = this->getPartitionID(source_id); std::lock_guard lock(*this->partitions[partition_id].mu); - if (this->is_spill_triggered) - { - // TODO need refinement - // auto spilled_block_num = static_cast(this->cte_spill.blockNum()); - // if (idx < spilled_block_num) - // return CTEOpStatus::IOIn; + if (this->partitions[partition_id].isBlockAvailableInDiskNoLock(cte_reader_id)) + return CTEOpStatus::IO_IN; - } - - - auto status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); + auto status = this->checkBlockAvailableInMemoryNoLock(cte_reader_id, partition_id); if (status != CTEOpStatus::OK) return status; - auto idx = this->partitions[partition_id].fetch_block_idxs[cte_reader_id].idx++; + auto idx = this->partitions[partition_id].getIdxInMemoryNoLock(cte_reader_id); block = this->partitions[partition_id].blocks[idx]; return status; } -CTEOpStatus CTE::checkBlockAvailableNoLock(size_t cte_reader_id, size_t partition_id) +CTEOpStatus CTE::checkBlockAvailableInMemoryNoLock(size_t cte_reader_id, size_t partition_id) { - { - std::shared_lock status_lock(this->aux_rw_lock); - if (this->cte_status != CTEStatus::NORMAL) - return CTEOpStatus::BLOCK_NOT_AVAILABLE; - } - - if unlikely (this->is_cancelled) - return CTEOpStatus::CANCELLED; - - if (this->is_spill_triggered) - { - // TODO judge if block is in the disk - // auto spilled_block_num = static_cast(this->cte_spill.blockNum()); - // if (idx < spilled_block_num) - // return CTEOpStatus::OK; - - // idx -= spilled_block_num; - } - - if (this->partitions[partition_id].blocks.size() - <= this->partitions[partition_id].fetch_block_idxs[cte_reader_id].idx) + if (!this->partitions[partition_id].isBlockAvailableInMemoryNoLock(cte_reader_id)) return this->is_eof ? CTEOpStatus::END_OF_FILE : CTEOpStatus::BLOCK_NOT_AVAILABLE; return CTEOpStatus::OK; @@ -88,57 +62,53 @@ CTEOpStatus CTE::checkBlockAvailableNoLock(size_t cte_reader_id, size_t partitio CTEOpStatus CTE::pushBlock(size_t sink_id, const Block & block) { - auto partition_id = this->getPartitionID(sink_id); - CTEOpStatus ret = CTEOpStatus::OK; + if unlikely (block.rows() == 0) + return CTEOpStatus::OK; + { - std::unique_lock status_lock(this->aux_rw_lock); - if (this->cte_status != CTEStatus::NORMAL) - { - if likely (block.rows() != 0) - // Block memory usage will be calculated after the finish of spill - this->partitions[partition_id].tmp_blocks.push_back(block); - return CTEOpStatus::IO_OUT; - } + std::shared_lock rw_lock(this->rw_lock); + if unlikely (this->is_cancelled) + return CTEOpStatus::CANCELLED; } - // This function is called in cpu pool, we don't want to wait for this lock too long. - // This lock may be held when spill is in execution. So we need ensure that cte status is not changed - std::unique_lock rw_lock(this->rw_lock); - - if unlikely (this->is_cancelled) - return CTEOpStatus::CANCELLED; + auto partition_id = this->getPartitionID(sink_id); + CTEOpStatus ret = CTEOpStatus::OK; - if unlikely (block.rows() == 0) - // All rows in block may have been filtered and it's needles to store this block - return CTEOpStatus::OK; + std::unique_lock status_lock(this->partitions[partition_id].aux_lock); + if (this->partitions[partition_id].status != CTEPartitionStatus::NORMAL) + { + if likely (block.rows() != 0) + // Block memory usage will be calculated after the finish of spill + this->partitions[partition_id].tmp_blocks.push_back(block); + return CTEOpStatus::IO_OUT; + } + // mu must be held after aux_lock so that we will not be blocked when spill is triggered. + // Blocked in cpu pool is very bad. std::lock_guard lock(*this->partitions[partition_id].mu); + this->partitions[partition_id].memory_usage += block.bytes(); this->partitions[partition_id].blocks.push_back(block); this->partitions[partition_id].pipe_cv->notifyOne(); - // TODO memory usage judgement should be considered from global perspective - // if (this->partitions[partition_id].memory_usage >= this->memory_threshold) - // { - // this->cte_status = CTE::NEED_SPILL; - // ret = CTEOpStatus::IO_OUT; - // } + if unlikely (this->partitions[partition_id].exceedMemoryThresholdNoLock()) + { + this->partitions[partition_id].setCTEPartitionStatusNoLock(CTEPartitionStatus::NEED_SPILL); + return CTEOpStatus::IO_OUT; + } return ret; } -CTEOpStatus CTE::getBlockFromDisk(size_t source_id, size_t idx, Block & block) +CTEOpStatus CTE::getBlockFromDisk(size_t cte_reader_id, size_t source_id, Block & block) { - auto partition_id = this->getPartitionID(source_id); - - std::shared_lock lock(this->rw_lock); - if unlikely (this->is_cancelled) + { + std::shared_lock lock(this->rw_lock); + if unlikely (this->is_cancelled) return CTEOpStatus::CANCELLED; + } + + auto partition_id = this->getPartitionID(source_id); - // We can call this function only when spill is triggered - RUNTIME_CHECK_MSG(this->is_spill_triggered, "Spill should be triggered"); - - // TODO - RUNTIME_CHECK_MSG(static_cast(this->cte_spill.blockNum()) <= idx, "Requested block is not in disk"); // TODO // block = this->cte_spill.readBlockAt(idx); @@ -149,10 +119,11 @@ CTEOpStatus CTE::getBlockFromDisk(size_t source_id, size_t idx, Block & block) bool CTE::spillBlocks(size_t sink_id) { auto partition_id = this->getPartitionID(sink_id); - std::unique_lock lock(this->rw_lock); - - if unlikely (this->is_cancelled) - return false; + { + std::shared_lock lock(this->rw_lock); + if unlikely (this->is_cancelled) + return false; + } while (true) { @@ -190,7 +161,7 @@ void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_ std::shared_lock rw_lock(this->rw_lock); std::lock_guard lock(*this->partitions[partition_id].mu); - status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); + status = this->checkBlockAvailableInMemoryNoLock(cte_reader_id, partition_id); if (status == CTEOpStatus::BLOCK_NOT_AVAILABLE) { @@ -200,10 +171,4 @@ void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_ this->notifyTaskDirectly(partition_id, std::move(task)); } - -CTE::CTEStatus CTE::getStatus() -{ - std::shared_lock lock(this->aux_rw_lock); - return this->cte_status; -} } // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index afefa9bcda5..66291ae53cc 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -16,56 +16,17 @@ #include #include -#include #include -#include #include -#include #include +#include -#include #include #include -#include #include namespace DB { -enum class CTEOpStatus -{ - OK, - BLOCK_NOT_AVAILABLE, // It means that we do not have specified block so far - IO_OUT, - IO_IN, - END_OF_FILE, - CANCELLED -}; - -struct IdxWithPadding -{ - IdxWithPadding() = default; - explicit IdxWithPadding(size_t idx_) - : idx(idx_) - {} - - size_t idx = 0; - - // To avoid false sharing - char padding[ABSL_CACHELINE_SIZE]{}; -}; - -struct CTEPartition -{ - std::unique_ptr mu; - Blocks blocks; - std::unordered_map fetch_block_idxs; // TODO id needs to be adjusted after some blocks have been spilled to the disk - size_t memory_usage = 0; // TODO need a unified statistic - std::unique_ptr pipe_cv; - - // TODO handle this, some blocks can not be spilled when spill is in execution, they can only be stored temporary - Blocks tmp_blocks; -}; - class CTE { public: @@ -80,13 +41,6 @@ class CTE } } - enum CTEStatus - { - NORMAL = 0, - NEED_SPILL, - IN_SPILLING, - }; - size_t getCTEReaderID() { std::unique_lock lock(this->rw_lock); @@ -109,8 +63,6 @@ class CTE return this->err_msg; } - CTEStatus getStatus(); - void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t source_id); void registerTask(size_t partition_id, TaskPtr && task, NotifyType type); @@ -119,7 +71,7 @@ class CTE this->partitions[partition_id].pipe_cv->notifyTaskDirectly(std::move(task)); } - CTEOpStatus getBlockFromDisk(size_t source_id, size_t idx, Block & block); + CTEOpStatus getBlockFromDisk(size_t cte_reader_id, size_t source_id, Block & block); bool spillBlocks(size_t sink_id); void addResp(const tipb::SelectResponse & resp) @@ -141,7 +93,8 @@ class CTE private: size_t getPartitionID(size_t id) const { return id % this->partition_num; } - CTEOpStatus checkBlockAvailableNoLock(size_t cte_reader_id, size_t partition_id); + // rw_lock and mu of partition need to be locked + CTEOpStatus checkBlockAvailableInMemoryNoLock(size_t cte_reader_id, size_t partition_id); void notifyImpl(bool is_eof, const String & msg) { @@ -164,15 +117,7 @@ class CTE size_t partition_num; std::vector partitions; - size_t memory_threshold = 0; - - CTEStatus cte_status = CTEStatus::NORMAL; - - // Protecting cte_status and tmp_blocks - std::shared_mutex aux_rw_lock; - - CTESpill cte_spill; - bool is_spill_triggered = false; + CTESpillContext cte_spill_context; std::shared_mutex rw_lock; bool is_eof = false; diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp new file mode 100644 index 00000000000..29037a458a0 --- /dev/null +++ b/dbms/src/Operators/CTEPartition.cpp @@ -0,0 +1,82 @@ +#include +#include +#include +#include +#include + +namespace DB +{ +size_t CTEPartition::getIdxInMemoryNoLock(size_t cte_reader_id) +{ + if (this->total_block_in_disk_num >= this->fetch_block_idxs[cte_reader_id].idx) + return this->fetch_block_idxs[cte_reader_id].idx; + return this->fetch_block_idxs[cte_reader_id].idx - this->total_block_in_disk_num; +} + +void CTEPartition::spillBlocks() +{ + std::lock_guard lock(*(this->mu)); + + for (const auto & block : this->tmp_blocks) + this->blocks.push_back(block); + this->tmp_blocks.clear(); + + auto cte_reader_num = this->fetch_block_idxs.size(); + std::vector split_idxs{0}; + split_idxs.reserve(cte_reader_num+1); + for (auto iter : this->fetch_block_idxs) + split_idxs.push_back(iter.second.idx); + std::sort(split_idxs.begin(), split_idxs.end()); + + auto begin_iter = this->blocks.begin(); + Blocks spilled_blocks; + auto idx_num = split_idxs.size(); + for (size_t i = 0; i < idx_num; i++) + { + if (split_idxs[i] >= this->blocks.size()) + break; + + if (i == idx_num - 1) + spilled_blocks.assign(begin_iter+split_idxs[i], this->blocks.end()); + else + spilled_blocks.assign(begin_iter+split_idxs[i], begin_iter+split_idxs[i+1]); + + + } +} + +void CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) +{ + std::lock_guard lock(*(this->mu)); + RUNTIME_CHECK_MSG(this->isSpillTriggeredNoLock(), "Spill should be triggered"); + RUNTIME_CHECK_MSG(this->isBlockAvailableInDiskNoLock(cte_reader_id), "Requested block is not in disk"); + + bool retry = false; + while (true) + { + auto [iter, _] = this->cte_reader_restore_streams.insert(std::make_pair(cte_reader_id, nullptr)); + if (iter->second == nullptr) + { + auto spiller_iter = this->spillers.find(cte_reader_id); + RUNTIME_CHECK_MSG(spiller_iter == this->spillers.end(), "cte reader {} can't find spiller", cte_reader_id); + auto streams = spiller_iter->second->restoreBlocks(this->partition_id, 1); + RUNTIME_CHECK(streams.size() == 1); + iter->second = streams[0]; + iter->second->readPrefix(); + } + + block = iter->second->read(); + if (!block) + { + RUNTIME_CHECK(!retry); + + iter->second->readSuffix(); + iter->second = nullptr; + retry = true; + continue; + } + + this->addIdxNoLock(cte_reader_id); + } +} +} // namespace DB diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h new file mode 100644 index 00000000000..304b2483bfd --- /dev/null +++ b/dbms/src/Operators/CTEPartition.h @@ -0,0 +1,94 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ +enum CTEPartitionStatus +{ + NORMAL = 0, + NEED_SPILL, + IN_SPILLING, +}; + +enum class CTEOpStatus +{ + OK, + BLOCK_NOT_AVAILABLE, // It means that we do not have specified block so far + IO_OUT, + IO_IN, + END_OF_FILE, + CANCELLED +}; + +struct IdxWithPadding +{ + IdxWithPadding() = default; + explicit IdxWithPadding(size_t idx_) + : idx(idx_) + {} + + size_t idx = 0; + + // To avoid false sharing + char padding[ABSL_CACHELINE_SIZE]{}; +}; + +struct CTEPartition +{ + size_t getIdxInMemoryNoLock(size_t cte_reader_id); + bool isBlockAvailableInDiskNoLock(size_t cte_reader_id) { return this->fetch_block_idxs[cte_reader_id].idx < this->total_block_in_disk_num; } + bool isBlockAvailableInMemoryNoLock(size_t cte_reader_id) { return this->getIdxInMemoryNoLock(cte_reader_id) < this->blocks.size(); } + bool exceedMemoryThresholdNoLock() const { return this->memory_usage >= this->memory_threoshold; } + void setCTEPartitionStatusNoLock(CTEPartitionStatus status) { this->status = status; } + bool isSpillTriggeredNoLock() const { return this->total_block_in_disk_num > 0; } + void addIdxNoLock(size_t cte_reader_id) { this->fetch_block_idxs[cte_reader_id].idx++; } + + void spillBlocks(); + void getBlockFromDisk(size_t cte_reader_id, Block & block); + + size_t partition_id; // TODO initialize it + + std::unique_ptr mu; + Blocks blocks; + std::unordered_map fetch_block_idxs; + size_t memory_usage = 0; + size_t memory_threoshold = 0; + std::unique_ptr pipe_cv; + + // Protecting cte_status and tmp_blocks + std::mutex aux_lock; + CTEPartitionStatus status; + // TODO handle this, some blocks can not be spilled when spill is in execution, they can only be stored temporary + Blocks tmp_blocks; + + std::vector block_in_disk_nums; + std::unordered_map spillers; + std::unordered_map cte_reader_restore_streams; + UInt64 total_block_in_disk_num = 0; + + CTESpillContext * spill_context; +}; +} // namespace DB From bff95e3d32473e2507254cc484c97db878924b5d Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 27 Jun 2025 16:13:50 +0800 Subject: [PATCH 075/118] save --- dbms/src/Flash/Mpp/CTEManager.cpp | 5 ++- dbms/src/Operators/CTE.cpp | 11 ++----- dbms/src/Operators/CTE.h | 55 +++++++++---------------------- dbms/src/Operators/CTEPartition.h | 54 ++++++++++++++++++++++++++++++ 4 files changed, 76 insertions(+), 49 deletions(-) create mode 100644 dbms/src/Operators/CTEPartition.h diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index edb177467ed..69efc70a760 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -71,6 +71,9 @@ std::shared_ptr CTEManager::getCTEImpl( this->ctes.insert(std::make_pair( query_id_and_cte_id, CTEWithCounter(std::make_shared(concurrency), expected_sink_num, expected_source_num))); - return this->ctes.find(query_id_and_cte_id)->second.getCTE(); + + auto cte = this->ctes.find(query_id_and_cte_id)->second.getCTE(); + cte->checkPartitionNum(concurrency); + return cte; } } // namespace DB diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index e8517101ebe..360e5412216 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include #include @@ -21,10 +20,8 @@ namespace DB { -CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & block) +CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block & block) { - auto partition_id = this->getPartitionID(source_id); - std::shared_lock rw_lock(this->rw_lock); std::lock_guard lock(*this->partitions[partition_id].mu); auto status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); @@ -36,7 +33,7 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & b return status; } -bool CTE::pushBlock(size_t sink_id, const Block & block) +bool CTE::pushBlock(size_t partition_id, const Block & block) { { std::shared_lock rw_lock(this->rw_lock); @@ -47,7 +44,6 @@ bool CTE::pushBlock(size_t sink_id, const Block & block) if unlikely (block.rows() == 0) return true; - auto partition_id = this->getPartitionID(sink_id); std::lock_guard lock(*this->partitions[partition_id].mu); this->partitions[partition_id].memory_usages += block.bytes(); this->partitions[partition_id].blocks.push_back(block); @@ -61,9 +57,8 @@ void CTE::registerTask(size_t partition_id, TaskPtr && task, NotifyType type) this->partitions[partition_id].pipe_cv->registerTask(std::move(task)); } -void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t source_id) +void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t partition_id) { - auto partition_id = this->getPartitionID(source_id); CTEOpStatus status; std::shared_lock rw_lock(this->rw_lock); diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index d67f266c34d..2eed3c1334f 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -14,51 +14,19 @@ #pragma once +#include #include #include -#include -#include -#include +#include #include #include #include #include -#include #include namespace DB { -enum class CTEOpStatus -{ - OK, - BLOCK_NOT_AVAILABLE, - END_OF_FILE, - CANCELLED -}; - -struct IdxWithPadding -{ - IdxWithPadding() = default; - explicit IdxWithPadding(size_t idx_) - : idx(idx_) - {} - - size_t idx = 0; - - // To avoid false sharing - char padding[ABSL_CACHELINE_SIZE]{}; -}; - -struct CTEPartition -{ - std::unique_ptr mu; - Blocks blocks; - std::unordered_map fetch_block_idxs; - size_t memory_usages = 0; // TODO need a unified statistic - std::unique_ptr pipe_cv; -}; - class CTE { public: @@ -73,6 +41,15 @@ class CTE } } + void checkPartitionNum(size_t partition_num) const + { + RUNTIME_CHECK_MSG( + this->partition_num == partition_num, + "expect partition num: {}, actual: {}", + this->partition_num, + partition_num); + } + size_t getCTEReaderID() { std::unique_lock lock(this->rw_lock); @@ -83,9 +60,9 @@ class CTE return cte_reader_id; } - CTEOpStatus tryGetBlockAt(size_t cte_reader_id, size_t source_id, Block & block); + CTEOpStatus tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block & block); - bool pushBlock(size_t sink_id, const Block & block); + bool pushBlock(size_t partition_id, const Block & block); void notifyEOF() { this->notifyImpl(true, ""); } void notifyCancel(const String & msg) { this->notifyImpl(false, msg); } @@ -95,7 +72,7 @@ class CTE return this->err_msg; } - void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t source_id); + void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t partition_id); void registerTask(size_t partition_id, TaskPtr && task, NotifyType type); void notifyTaskDirectly(size_t partition_id, TaskPtr && task) @@ -120,8 +97,6 @@ class CTE } private: - size_t getPartitionID(size_t id) const { return id % this->partition_num; } - CTEOpStatus checkBlockAvailableNoLock(size_t cte_reader_id, size_t partition_id) { if unlikely (this->is_cancelled) @@ -150,7 +125,7 @@ class CTE partition.pipe_cv->notifyAll(); } - size_t partition_num; + const size_t partition_num; std::vector partitions; std::shared_mutex rw_lock; diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h new file mode 100644 index 00000000000..354885f0109 --- /dev/null +++ b/dbms/src/Operators/CTEPartition.h @@ -0,0 +1,54 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ +enum class CTEOpStatus +{ + OK, + BLOCK_NOT_AVAILABLE, + END_OF_FILE, + CANCELLED +}; + +struct IdxWithPadding +{ + IdxWithPadding() = default; + explicit IdxWithPadding(size_t idx_) + : idx(idx_) + {} + + size_t idx = 0; + + // To avoid false sharing + char padding[ABSL_CACHELINE_SIZE]{}; +}; + +struct CTEPartition +{ + std::unique_ptr mu; + Blocks blocks; + std::unordered_map fetch_block_idxs; + size_t memory_usages = 0; + std::unique_ptr pipe_cv; +}; +} // namespace DB From c32757c23c36a8cb590b1c228e84200276ce887b Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 1 Jul 2025 10:58:04 +0800 Subject: [PATCH 076/118] remove IdxWithPadding --- dbms/src/Operators/CTE.cpp | 2 +- dbms/src/Operators/CTE.h | 4 ++-- dbms/src/Operators/CTEPartition.h | 16 +--------------- 3 files changed, 4 insertions(+), 18 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 360e5412216..1b51e959e28 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -28,7 +28,7 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block if (status != CTEOpStatus::OK) return status; - auto idx = this->partitions[partition_id].fetch_block_idxs[cte_reader_id].idx++; + auto idx = this->partitions[partition_id].fetch_block_idxs[cte_reader_id]++; block = this->partitions[partition_id].blocks[idx]; return status; } diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 2eed3c1334f..8253531029d 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -56,7 +56,7 @@ class CTE auto cte_reader_id = this->next_cte_reader_id; this->next_cte_reader_id++; for (auto & item : this->partitions) - item.fetch_block_idxs.insert(std::make_pair(cte_reader_id, IdxWithPadding(0))); + item.fetch_block_idxs.insert(std::make_pair(cte_reader_id, 0)); return cte_reader_id; } @@ -103,7 +103,7 @@ class CTE return CTEOpStatus::CANCELLED; if (this->partitions[partition_id].blocks.size() - <= this->partitions[partition_id].fetch_block_idxs[cte_reader_id].idx) + <= this->partitions[partition_id].fetch_block_idxs[cte_reader_id]) return this->is_eof ? CTEOpStatus::END_OF_FILE : CTEOpStatus::BLOCK_NOT_AVAILABLE; return CTEOpStatus::OK; diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 354885f0109..5e9436f12ef 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -16,7 +16,6 @@ #include #include -#include #include @@ -30,24 +29,11 @@ enum class CTEOpStatus CANCELLED }; -struct IdxWithPadding -{ - IdxWithPadding() = default; - explicit IdxWithPadding(size_t idx_) - : idx(idx_) - {} - - size_t idx = 0; - - // To avoid false sharing - char padding[ABSL_CACHELINE_SIZE]{}; -}; - struct CTEPartition { std::unique_ptr mu; Blocks blocks; - std::unordered_map fetch_block_idxs; + std::unordered_map fetch_block_idxs; size_t memory_usages = 0; std::unique_ptr pipe_cv; }; From d832edd5b7f1423ac3f3273a7ae13be19340fbc7 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 1 Jul 2025 17:28:39 +0800 Subject: [PATCH 077/118] save --- dbms/src/Common/TiFlashMetrics.h | 2 +- dbms/src/Interpreters/CTESpillContext.cpp | 8 +- dbms/src/Interpreters/CTESpillContext.h | 12 +- dbms/src/Operators/CTE.cpp | 151 +++++++--------------- dbms/src/Operators/CTE.h | 32 ++--- dbms/src/Operators/CTEPartition.cpp | 133 +++++++++++++++++-- dbms/src/Operators/CTEPartition.h | 51 ++++++-- dbms/src/Operators/CTEReader.cpp | 13 +- dbms/src/Operators/CTEReader.h | 6 +- dbms/src/Operators/CTESinkOp.cpp | 2 +- dbms/src/Operators/CTESourceOp.cpp | 5 +- dbms/src/Operators/CTESourceOp.h | 8 -- 12 files changed, 238 insertions(+), 185 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 6aaf82626fe..fbac79262af 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -778,7 +778,7 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_wait_on_join_build, {"type", "wait_on_join_build"}), \ F(type_wait_on_join_probe, {"type", "wait_on_join_probe"}), \ F(type_wait_on_result_queue_write, {"type", "wait_on_result_queue_write"}), \ - F(type_wait_on_cte, {"type", "wait_on_cte"})) \ + F(type_wait_on_cte, {"type", "wait_on_cte"})) \ M(tiflash_pipeline_task_duration_seconds, \ "Bucketed histogram of pipeline task duration in seconds", \ Histogram, /* these command usually cost several hundred milliseconds to several seconds, increase the start bucket to 5ms */ \ diff --git a/dbms/src/Interpreters/CTESpillContext.cpp b/dbms/src/Interpreters/CTESpillContext.cpp index e2ac9f1d9b2..43eb4aec02f 100644 --- a/dbms/src/Interpreters/CTESpillContext.cpp +++ b/dbms/src/Interpreters/CTESpillContext.cpp @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include +#include #include @@ -24,13 +24,13 @@ SpillerSharedPtr CTESpillContext::getSpillAt(size_t idx) std::lock_guard lock(this->mu); auto spiller_num = this->spillers.size(); - // The spiller whose idx is lower that the parameter idx must have been created + // The spiller whose idx is lower that the parameter idx must have been created before RUNTIME_CHECK_MSG(idx <= spiller_num, "idx: {}, spiller_num: {}", idx, spiller_num); if (idx < spiller_num) return this->spillers[idx]; - - + + // TODO create new spiller return this->spillers[idx]; } } // namespace DB diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h index ea7f978fb6a..dced5223a86 100644 --- a/dbms/src/Interpreters/CTESpillContext.h +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -14,10 +14,10 @@ #pragma once -#include -#include #include #include +#include +#include #include namespace DB @@ -30,13 +30,13 @@ class CTESpillContext final : public OperatorSpillContext const Block & spill_block_schema_, UInt64 operator_spill_threshold_, const LoggerPtr & log_) - : OperatorSpillContext(operator_spill_threshold_, "cte", log_) - , spill_config(spill_config_) - , spill_block_schema(spill_block_schema_) + : OperatorSpillContext(operator_spill_threshold_, "cte", log_) + , spill_config(spill_config_) + , spill_block_schema(spill_block_schema_) {} ~CTESpillContext() override = default; - + Int64 getTotalRevocableMemoryImpl() override { throw Exception(""); } // TODO implement bool supportFurtherSpill() const override { throw Exception(""); } // TODO implement bool supportAutoTriggerSpill() const override { throw Exception(""); } // TODO implement diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 0f856cb2ee9..de8b402cb1a 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -12,12 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -<<<<<<< HEAD #include #include #include -======= ->>>>>>> cte #include #include @@ -29,148 +26,90 @@ namespace DB { CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block & block) { - auto partition_id = this->getPartitionID(source_id); + std::shared_lock rw_lock(this->rw_lock); + if unlikely (this->is_cancelled) + return CTEOpStatus::CANCELLED; + auto status = this->partitions[partition_id].tryGetBlockAt(cte_reader_id, block); + switch (status) { - std::shared_lock rw_lock(this->rw_lock); - if unlikely (this->is_cancelled) - return CTEOpStatus::CANCELLED; - } - - std::lock_guard status_lock(this->partitions[partition_id].aux_lock); - if (this->partitions[partition_id].status != CTEPartitionStatus::NORMAL) - return CTEOpStatus::IO_OUT; - - std::lock_guard lock(*this->partitions[partition_id].mu); - - if (this->partitions[partition_id].isBlockAvailableInDiskNoLock(cte_reader_id)) - return CTEOpStatus::IO_IN; - - auto status = this->checkBlockAvailableInMemoryNoLock(cte_reader_id, partition_id); - if (status != CTEOpStatus::OK) + case CTEOpStatus::BLOCK_NOT_AVAILABLE: + return this->is_eof ? CTEOpStatus::END_OF_FILE : CTEOpStatus::BLOCK_NOT_AVAILABLE; + default: return status; - - auto idx = this->partitions[partition_id].getIdxInMemoryNoLock(cte_reader_id); - block = this->partitions[partition_id].blocks[idx]; - return status; + } } -CTEOpStatus CTE::checkBlockAvailableInMemoryNoLock(size_t cte_reader_id, size_t partition_id) +CTEOpStatus CTE::pushBlock(size_t partition_id, const Block & block) { - if (!this->partitions[partition_id].isBlockAvailableInMemoryNoLock(cte_reader_id)) - return this->is_eof ? CTEOpStatus::END_OF_FILE : CTEOpStatus::BLOCK_NOT_AVAILABLE; + if unlikely (block.rows() == 0) + return CTEOpStatus::OK; - return CTEOpStatus::OK; + std::shared_lock rw_lock(this->rw_lock); + if unlikely (this->is_cancelled) + return CTEOpStatus::CANCELLED; + + return this->partitions[partition_id].pushBlock(block); } -CTEOpStatus CTE::pushBlock(size_t sink_id, const Block & block) +CTEOpStatus CTE::getBlockFromDisk(size_t cte_reader_id, size_t partition_id, Block & block) { - if unlikely (block.rows() == 0) - return CTEOpStatus::OK; - { - std::shared_lock rw_lock(this->rw_lock); + std::shared_lock lock(this->rw_lock); if unlikely (this->is_cancelled) return CTEOpStatus::CANCELLED; } - auto partition_id = this->getPartitionID(sink_id); - CTEOpStatus ret = CTEOpStatus::OK; - - std::unique_lock status_lock(this->partitions[partition_id].aux_lock); - if (this->partitions[partition_id].status != CTEPartitionStatus::NORMAL) - { - if likely (block.rows() != 0) - // Block memory usage will be calculated after the finish of spill - this->partitions[partition_id].tmp_blocks.push_back(block); - return CTEOpStatus::IO_OUT; - } - - // mu must be held after aux_lock so that we will not be blocked when spill is triggered. - // Blocked in cpu pool is very bad. - std::lock_guard lock(*this->partitions[partition_id].mu); - - this->partitions[partition_id].memory_usage += block.bytes(); - this->partitions[partition_id].blocks.push_back(block); - this->partitions[partition_id].pipe_cv->notifyOne(); - - if unlikely (this->partitions[partition_id].exceedMemoryThresholdNoLock()) - { - this->partitions[partition_id].setCTEPartitionStatusNoLock(CTEPartitionStatus::NEED_SPILL); - return CTEOpStatus::IO_OUT; - } - return ret; + return this->partitions[partition_id].getBlockFromDisk(cte_reader_id, block); } -CTEOpStatus CTE::getBlockFromDisk(size_t cte_reader_id, size_t source_id, Block & block) +CTEOpStatus CTE::spillBlocks(size_t partition_id) { { std::shared_lock lock(this->rw_lock); if unlikely (this->is_cancelled) - return CTEOpStatus::CANCELLED; + return CTEOpStatus::CANCELLED; } - - auto partition_id = this->getPartitionID(source_id); - - // TODO - // block = this->cte_spill.readBlockAt(idx); - return CTEOpStatus::OK; + return this->partitions[partition_id].spillBlocks(); } -// TODO maybe CTEPartition should also implement a similiar function -bool CTE::spillBlocks(size_t sink_id) +void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t partition_id) { - auto partition_id = this->getPartitionID(sink_id); + std::shared_lock rw_lock(this->rw_lock); + if (this->is_cancelled) { - std::shared_lock lock(this->rw_lock); - if unlikely (this->is_cancelled) - return false; + this->notifyTaskDirectly(partition_id, std::move(task)); + return; } - while (true) + std::lock_guard status_lock(*(this->partitions[partition_id].status_lock)); + if (this->partitions[partition_id].status == CTEPartitionStatus::IN_SPILLING) { - this->cte_spill.writeBlocks(this->partitions[partition_id].blocks); - this->partitions[partition_id].blocks.clear(); - this->partitions[partition_id].memory_usage = 0; - - std::unique_lock aux_lock(this->aux_rw_lock); - for (const auto & block : this->partitions[partition_id].tmp_blocks) - { - this->partitions[partition_id].blocks.push_back(block); - this->partitions[partition_id].memory_usage += block.bytes(); - } - - this->partitions[partition_id].tmp_blocks.clear(); - - // TODO we need to consider total memory usage, not partition memory usage - if (this->partitions[partition_id].memory_usage < this->memory_threshold) - { - this->cte_status = CTEStatus::NORMAL; - break; - } + this->notifyTaskDirectly(partition_id, std::move(task)); + return; } - // Many tasks may be waiting for the finish of spill - this->partitions[partition_id].pipe_cv->notifyAll(); - return true; + std::lock_guard lock(*(this->partitions[partition_id].mu)); + if (this->partitions[partition_id].isBlockAvailableNoLock(cte_reader_id) || this->is_eof) + this->notifyTaskDirectly(partition_id, std::move(task)); + else + this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); } -// TODO sometimes we register task because of spill, consider this situation -void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t source_id) +void CTE::checkInSpillingAndRegisterTask(TaskPtr && task, size_t partition_id) { - CTEOpStatus status; - std::shared_lock rw_lock(this->rw_lock); - std::lock_guard lock(*this->partitions[partition_id].mu); - status = this->checkBlockAvailableInMemoryNoLock(cte_reader_id, partition_id); - - if (status == CTEOpStatus::BLOCK_NOT_AVAILABLE) + if (this->is_cancelled) { - this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); + this->notifyTaskDirectly(partition_id, std::move(task)); return; } - this->notifyTaskDirectly(partition_id, std::move(task)); + std::lock_guard status_lock(*(this->partitions[partition_id].status_lock)); + if (this->partitions[partition_id].status == CTEPartitionStatus::IN_SPILLING) + this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); + else + this->notifyTaskDirectly(partition_id, std::move(task)); } } // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 4d4e4b956e0..6e98f3d7058 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -19,8 +19,8 @@ #include #include #include -#include #include +#include #include #include @@ -35,11 +35,7 @@ class CTE : partition_num(partition_num_) { for (size_t i = 0; i < this->partition_num; i++) - { - this->partitions.push_back(CTEPartition()); - this->partitions.back().mu = std::make_unique(); - this->partitions.back().pipe_cv = std::make_unique(); - } + this->partitions.push_back(CTEPartition(i, &(this->cte_spill_context))); } void checkPartitionNum(size_t partition_num) const @@ -62,8 +58,7 @@ class CTE } CTEOpStatus tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block & block); - - CTEOpStatus pushBlock(size_t sink_id, const Block & block); + CTEOpStatus pushBlock(size_t partition_id, const Block & block); void notifyEOF() { this->notifyImpl(true, ""); } void notifyCancel(const String & msg) { this->notifyImpl(false, msg); } @@ -74,16 +69,22 @@ class CTE } void checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t partition_id); + void checkInSpillingAndRegisterTask(TaskPtr && task, size_t partition_id); + + CTEOpStatus getBlockFromDisk(size_t cte_reader_id, size_t partition_id, Block & block); + CTEOpStatus spillBlocks(size_t partition_id); + + void registerTask(size_t partition_id, TaskPtr && task, NotifyType type) + { + task->setNotifyType(type); + this->partitions[partition_id].pipe_cv->registerTask(std::move(task)); + } - void registerTask(size_t partition_id, TaskPtr && task, NotifyType type); void notifyTaskDirectly(size_t partition_id, TaskPtr && task) { this->partitions[partition_id].pipe_cv->notifyTaskDirectly(std::move(task)); } - CTEOpStatus getBlockFromDisk(size_t cte_reader_id, size_t source_id, Block & block); - bool spillBlocks(size_t sink_id); - void addResp(const tipb::SelectResponse & resp) { std::unique_lock lock(this->rw_lock); @@ -101,11 +102,6 @@ class CTE } private: - size_t getPartitionID(size_t id) const { return id % this->partition_num; } - - // rw_lock and mu of partition need to be locked - CTEOpStatus checkBlockAvailableInMemoryNoLock(size_t cte_reader_id, size_t partition_id); - void notifyImpl(bool is_eof, const String & msg) { std::unique_lock lock(this->rw_lock); @@ -127,7 +123,7 @@ class CTE size_t partition_num; std::vector partitions; - CTESpillContext cte_spill_context; + CTESpillContext cte_spill_context; // TODO initialize it std::shared_mutex rw_lock; bool is_eof = false; diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 044ea31e133..4a1d085dd71 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -1,8 +1,23 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include #include + #include #include #include -#include namespace DB { @@ -13,9 +28,74 @@ size_t CTEPartition::getIdxInMemoryNoLock(size_t cte_reader_id) return this->fetch_block_idxs[cte_reader_id] - this->total_block_in_disk_num; } -void CTEPartition::spillBlocks() +CTEOpStatus CTEPartition::tryGetBlockAt(size_t cte_reader_id, Block & block) +{ + std::lock_guard status_lock(*(this->status_lock)); + if (this->status == CTEPartitionStatus::IN_SPILLING) + return CTEOpStatus::IO_OUT; + + std::lock_guard lock(*this->mu); + + if (this->isBlockAvailableInDiskNoLock(cte_reader_id)) + return CTEOpStatus::IO_IN; + + if (!this->isBlockAvailableInMemoryNoLock(cte_reader_id)) + return CTEOpStatus::BLOCK_NOT_AVAILABLE; + + auto idx = this->getIdxInMemoryNoLock(cte_reader_id); + block = this->blocks[idx]; + return CTEOpStatus::OK; +} + +CTEOpStatus CTEPartition::pushBlock(const Block & block) { - std::lock_guard lock(*(this->mu)); + std::unique_lock status_lock(*(this->status_lock)); + if (this->status != CTEPartitionStatus::NORMAL) + { + if likely (block.rows() != 0) + { + std::lock_guard lock(*this->mu); + + // Block memory usage will be calculated after the finish of spill + this->tmp_blocks.push_back(block); + } + return CTEOpStatus::IO_OUT; + } + + // mu must be held after status_lock so that we will not be blocked when spill is triggered. + // Blocked in cpu pool is very bad. + std::lock_guard lock(*this->mu); + + this->memory_usage += block.bytes(); + this->blocks.push_back(block); + this->pipe_cv->notifyOne(); + + if unlikely (this->exceedMemoryThresholdNoLock()) + { + this->setCTEPartitionStatusNoLock(CTEPartitionStatus::NEED_SPILL); + return CTEOpStatus::IO_OUT; + } + return CTEOpStatus::OK; +} + +CTEOpStatus CTEPartition::spillBlocks() +{ + std::unique_lock lock(*(this->mu), std::defer_lock); + { + std::lock_guard status_lock(*(this->status_lock)); + switch (this->status) + { + case CTEPartitionStatus::NORMAL: + return CTEOpStatus::OK; + case CTEPartitionStatus::IN_SPILLING: + return CTEOpStatus::IO_OUT; + case CTEPartitionStatus::NEED_SPILL: + this->setCTEPartitionStatusNoLock(CTEPartitionStatus::IN_SPILLING); + break; + } + + lock.lock(); + } for (const auto & block : this->tmp_blocks) this->blocks.push_back(block); @@ -23,36 +103,61 @@ void CTEPartition::spillBlocks() auto cte_reader_num = this->fetch_block_idxs.size(); std::vector split_idxs{0}; - split_idxs.reserve(cte_reader_num+1); + split_idxs.reserve(cte_reader_num + 1); for (auto iter : this->fetch_block_idxs) split_idxs.push_back(iter.second); std::sort(split_idxs.begin(), split_idxs.end()); auto begin_iter = this->blocks.begin(); - Blocks spilled_blocks; auto idx_num = split_idxs.size(); for (size_t i = 0; i < idx_num; i++) { if (split_idxs[i] >= this->blocks.size()) break; + Blocks spilled_blocks; if (i == idx_num - 1) - spilled_blocks.assign(begin_iter+split_idxs[i], this->blocks.end()); + spilled_blocks.assign(begin_iter + split_idxs[i], this->blocks.end()); else - spilled_blocks.assign(begin_iter+split_idxs[i], begin_iter+split_idxs[i+1]); + spilled_blocks.assign(begin_iter + split_idxs[i], begin_iter + split_idxs[i + 1]); - + auto spiller = this->spill_context->getSpillAt(i); + this->spillers.insert(std::make_pair(split_idxs[i], spiller)); + spiller->spillBlocks(std::move(spilled_blocks), this->partition_id); } + + this->blocks.clear(); + this->memory_usage = 0; + + std::lock_guard status_lock(*(this->status_lock)); + this->setCTEPartitionStatusNoLock(CTEPartitionStatus::NORMAL); + + // Many tasks may be waiting for the finish of spill + this->pipe_cv->notifyAll(); + return CTEOpStatus::OK; } -void CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) +CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) { - std::lock_guard lock(*(this->mu)); + std::unique_lock lock(*(this->mu), std::defer_lock); + { + std::lock_guard status_lock(*(this->status_lock)); + switch (this->status) + { + case CTEPartitionStatus::IN_SPILLING: + return CTEOpStatus::IO_OUT; + default: + break; + } + + lock.lock(); + } + RUNTIME_CHECK_MSG(this->isSpillTriggeredNoLock(), "Spill should be triggered"); RUNTIME_CHECK_MSG(this->isBlockAvailableInDiskNoLock(cte_reader_id), "Requested block is not in disk"); bool retry = false; - while (true) + do { auto [iter, _] = this->cte_reader_restore_streams.insert(std::make_pair(cte_reader_id, nullptr)); if (iter->second == nullptr) @@ -69,7 +174,7 @@ void CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) if (!block) { RUNTIME_CHECK(!retry); - + iter->second->readSuffix(); iter->second = nullptr; retry = true; @@ -77,6 +182,8 @@ void CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) } this->addIdxNoLock(cte_reader_id); - } + } while (retry); + + return CTEOpStatus::OK; } } // namespace DB diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index fe85d4d3fb9..3b11b912bc2 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -14,14 +14,14 @@ #pragma once -#include #include #include +#include #include +#include #include #include -#include namespace DB { @@ -44,34 +44,57 @@ enum class CTEOpStatus struct CTEPartition { + CTEPartition(size_t partition_id_, CTESpillContext * spill_context_) + : partition_id(partition_id_) + , mu(std::make_unique()) + , pipe_cv(std::make_unique()) + , status_lock(std::make_unique()) + , spill_context(spill_context_) + {} + size_t getIdxInMemoryNoLock(size_t cte_reader_id); - bool isBlockAvailableInDiskNoLock(size_t cte_reader_id) { return this->fetch_block_idxs[cte_reader_id] < this->total_block_in_disk_num; } - bool isBlockAvailableInMemoryNoLock(size_t cte_reader_id) { return this->getIdxInMemoryNoLock(cte_reader_id) < this->blocks.size(); } + bool isBlockAvailableInDiskNoLock(size_t cte_reader_id) + { + return this->fetch_block_idxs[cte_reader_id] < this->total_block_in_disk_num; + } + bool isBlockAvailableInMemoryNoLock(size_t cte_reader_id) + { + return this->getIdxInMemoryNoLock(cte_reader_id) < this->blocks.size(); + } bool exceedMemoryThresholdNoLock() const { return this->memory_usage >= this->memory_threoshold; } void setCTEPartitionStatusNoLock(CTEPartitionStatus status) { this->status = status; } bool isSpillTriggeredNoLock() const { return this->total_block_in_disk_num > 0; } void addIdxNoLock(size_t cte_reader_id) { this->fetch_block_idxs[cte_reader_id]++; } - void spillBlocks(); - void getBlockFromDisk(size_t cte_reader_id, Block & block); + CTEOpStatus pushBlock(const Block & block); + CTEOpStatus tryGetBlockAt(size_t cte_reader_id, Block & block); + CTEOpStatus spillBlocks(); + CTEOpStatus getBlockFromDisk(size_t cte_reader_id, Block & block); + + bool isBlockAvailableNoLock(size_t cte_reader_id) + { + if (this->isBlockAvailableInDiskNoLock(cte_reader_id)) + return true; + + return this->isBlockAvailableInMemoryNoLock(cte_reader_id); + } + + size_t partition_id; - size_t partition_id; // TODO initialize it - std::unique_ptr mu; Blocks blocks; std::unordered_map fetch_block_idxs; size_t memory_usage = 0; size_t memory_threoshold = 0; std::unique_ptr pipe_cv; - - // Protecting cte_status and tmp_blocks - std::mutex aux_lock; - CTEPartitionStatus status; - // TODO handle this, some blocks can not be spilled when spill is in execution, they can only be stored temporary Blocks tmp_blocks; + // Protecting cte_status + std::unique_ptr status_lock; + CTEPartitionStatus status = CTEPartitionStatus::NORMAL; + std::vector block_in_disk_nums; - std::unordered_map spillers; + std::unordered_map spillers; std::unordered_map cte_reader_restore_streams; UInt64 total_block_in_disk_num = 0; diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 03dd00b5242..cfbdab7cbdf 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -19,9 +19,9 @@ namespace DB { -CTEOpStatus CTEReader::fetchNextBlock(size_t source_id, Block & block) +CTEOpStatus CTEReader::fetchNextBlock(size_t partition_id, Block & block) { - auto ret = this->cte->tryGetBlockAt(this->cte_reader_id, source_id, block); + auto ret = this->cte->tryGetBlockAt(this->cte_reader_id, partition_id, block); switch (ret) { case CTEOpStatus::END_OF_FILE: @@ -41,13 +41,8 @@ CTEOpStatus CTEReader::fetchNextBlock(size_t source_id, Block & block) throw Exception("Should not reach here"); } -CTEOpStatus CTEReader::fetchBlockFromDisk(size_t source_id, Block & block) +CTEOpStatus CTEReader::fetchBlockFromDisk(size_t partition_id, Block & block) { - // TODO implement it - // std::lock_guard lock(this->mu); - // auto status = this->cte->getBlockFromDisk(this->block_fetch_idx, block); - // if likely (status == CTEOpStatus::Ok) - // this->block_fetch_idx++; - // return status; + return this->cte->getBlockFromDisk(this->cte_reader_id, partition_id, block); } } // namespace DB diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 436e0768653..41beb2b4dac 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -47,10 +47,8 @@ class CTEReader this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id); } - CTEOpStatus fetchNextBlock(size_t source_id, Block & block); - CTEOpStatus fetchBlockFromDisk(size_t source_id, Block & block); - - CTE::CTEStatus getCTEStatus() const { return this->cte->getStatus(); } + CTEOpStatus fetchNextBlock(size_t partition_id, Block & block); + CTEOpStatus fetchBlockFromDisk(size_t partition_id, Block & block); void getResp(tipb::SelectResponse & resp) { diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 941741895b1..7ea2dd49574 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include -#include "Operators/CTE.h" namespace DB { diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index fead74387f5..0e54fd57c42 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -49,7 +50,6 @@ OperatorStatus CTESourceOp::readImpl(Block & block) return OperatorStatus::IO_IN; case CTEOpStatus::IO_OUT: // CTE is spilling blocks to disk, we need to wait the finish of spill - this->wait_type = CTESourceOp::Spill; // TODO set corresponding notifier return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::CANCELLED: @@ -70,6 +70,9 @@ OperatorStatus CTESourceOp::executeIOImpl() { case CTEOpStatus::OK: return OperatorStatus::HAS_OUTPUT; + case CTEOpStatus::IO_OUT: + // TODO set notifier to wait for the finish of spill + return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::CANCELLED: return OperatorStatus::CANCELLED; default: diff --git a/dbms/src/Operators/CTESourceOp.h b/dbms/src/Operators/CTESourceOp.h index a734452d40c..7fbb923a1c7 100644 --- a/dbms/src/Operators/CTESourceOp.h +++ b/dbms/src/Operators/CTESourceOp.h @@ -56,7 +56,6 @@ class CTESourceOp : public SourceOp size_t id_, const NamesAndTypes & schema) : SourceOp(exec_context_, req_id) - , wait_type(NeedMoreBlock) , cte_reader(cte_reader_) , io_profile_info(IOProfileInfo::createForRemote(profile_info_ptr, 1)) , id(id_) @@ -75,17 +74,10 @@ class CTESourceOp : public SourceOp OperatorStatus executeIOImpl() override; private: - enum WaitType - { - NeedMoreBlock, - Spill, - }; - String query_id_and_cte_id; Block block_from_disk; uint64_t total_rows{}; - WaitType wait_type; std::shared_ptr cte_reader; IOProfileInfoPtr io_profile_info; From 0f6f8fa2fa79da7323caecae36393e007d0c17c8 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 1 Jul 2025 20:42:11 +0800 Subject: [PATCH 078/118] add io_notifier --- dbms/src/Operators/CTE.h | 18 ++++++++++++++++++ dbms/src/Operators/CTEPartition.h | 2 +- dbms/src/Operators/CTESinkOp.cpp | 24 ++++++++++++++++++++---- dbms/src/Operators/CTESinkOp.h | 3 ++- dbms/src/Operators/CTESourceOp.cpp | 5 +++-- dbms/src/Operators/CTESourceOp.h | 2 ++ 6 files changed, 46 insertions(+), 8 deletions(-) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 6e98f3d7058..ff286eef47d 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -134,4 +134,22 @@ class CTE String err_msg; }; + +class CTEIONotifier : public NotifyFuture +{ +public: + CTEIONotifier(std::shared_ptr cte_, size_t partition_id_) + : cte(cte_) + , partition_id(partition_id_) + {} + + void registerTask(TaskPtr && task) override + { + this->cte->checkInSpillingAndRegisterTask(std::move(task), this->partition_id); + } + +private: + std::shared_ptr cte; + size_t partition_id; +}; } // namespace DB diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 3b11b912bc2..70ed73795e3 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -85,7 +85,7 @@ struct CTEPartition Blocks blocks; std::unordered_map fetch_block_idxs; size_t memory_usage = 0; - size_t memory_threoshold = 0; + size_t memory_threoshold = 0; // TODO initialize it std::unique_ptr pipe_cv; Blocks tmp_blocks; diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index 7ea2dd49574..e2c9ed70986 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -17,6 +17,9 @@ #include #include +#include "Operators/CTEPartition.h" +#include "magic_enum.hpp" + namespace DB { void CTESinkOp::operateSuffixImpl() @@ -34,8 +37,9 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) switch (status) { case CTEOpStatus::IO_OUT: - // TODO set notifier - return OperatorStatus::IO_OUT; + // CTE is spilling blocks to disk, we need to wait the finish of spill + DB::setNotifyFuture(&(this->io_notifier)); + return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::OK: return OperatorStatus::NEED_INPUT; case CTEOpStatus::CANCELLED: @@ -47,8 +51,20 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) OperatorStatus CTESinkOp::executeIOImpl() { - if likely (this->cte->spillBlocks(this->id)) + auto status = this->cte->spillBlocks(this->id); + switch (status) + { + case CTEOpStatus::OK: return OperatorStatus::NEED_INPUT; - return OperatorStatus::CANCELLED; + case CTEOpStatus::IO_OUT: + // CTE is spilling blocks to disk, we need to wait the finish of spill + DB::setNotifyFuture(&(this->io_notifier)); + + return OperatorStatus::WAIT_FOR_NOTIFY; + case CTEOpStatus::CANCELLED: + return OperatorStatus::CANCELLED; + default: + throw Exception(fmt::format("Unexpected status {}", magic_enum::enum_name(status))); + } } } // namespace DB diff --git a/dbms/src/Operators/CTESinkOp.h b/dbms/src/Operators/CTESinkOp.h index 1136ae05fe8..f6db5d7cbf4 100644 --- a/dbms/src/Operators/CTESinkOp.h +++ b/dbms/src/Operators/CTESinkOp.h @@ -14,7 +14,6 @@ #pragma once -#include #include #include #include @@ -28,6 +27,7 @@ class CTESinkOp : public SinkOp : SinkOp(exec_context_, req_id) , cte(cte_) , id(id_) + , io_notifier(this->cte, id) {} String getName() const override { return "CTESinkOp"; } @@ -41,5 +41,6 @@ class CTESinkOp : public SinkOp std::shared_ptr cte; size_t total_rows = 0; size_t id; + CTEIONotifier io_notifier; }; } // namespace DB diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index 0e54fd57c42..eed4ddbe39f 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -50,7 +50,7 @@ OperatorStatus CTESourceOp::readImpl(Block & block) return OperatorStatus::IO_IN; case CTEOpStatus::IO_OUT: // CTE is spilling blocks to disk, we need to wait the finish of spill - // TODO set corresponding notifier + DB::setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::CANCELLED: return OperatorStatus::CANCELLED; @@ -71,7 +71,8 @@ OperatorStatus CTESourceOp::executeIOImpl() case CTEOpStatus::OK: return OperatorStatus::HAS_OUTPUT; case CTEOpStatus::IO_OUT: - // TODO set notifier to wait for the finish of spill + // CTE is spilling blocks to disk, we need to wait the finish of spill + DB::setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::CANCELLED: return OperatorStatus::CANCELLED; diff --git a/dbms/src/Operators/CTESourceOp.h b/dbms/src/Operators/CTESourceOp.h index 7fbb923a1c7..61404d3c9f9 100644 --- a/dbms/src/Operators/CTESourceOp.h +++ b/dbms/src/Operators/CTESourceOp.h @@ -60,6 +60,7 @@ class CTESourceOp : public SourceOp , io_profile_info(IOProfileInfo::createForRemote(profile_info_ptr, 1)) , id(id_) , notifier(this->cte_reader->getCTE(), this->cte_reader->getID(), this->id) + , io_notifier(this->cte_reader->getCTE(), id) { setHeader(Block(getColumnWithTypeAndName(schema))); } @@ -84,5 +85,6 @@ class CTESourceOp : public SourceOp tipb::SelectResponse resp; size_t id; CTESourceNotifyFuture notifier; + CTEIONotifier io_notifier; }; } // namespace DB From 19a3abc9b7bd11b340e028e2094aa9cf106e34aa Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 2 Jul 2025 16:08:18 +0800 Subject: [PATCH 079/118] fix hang --- dbms/src/Flash/Mpp/CTEManager.cpp | 4 ++-- dbms/src/Flash/Mpp/CTEManager.h | 22 ++----------------- dbms/src/Flash/Mpp/MPPTask.cpp | 1 - .../Flash/Planner/Plans/PhysicalCTESink.cpp | 2 +- dbms/src/Operators/CTE.cpp | 4 +--- dbms/src/Operators/CTEReader.h | 3 +-- 6 files changed, 7 insertions(+), 29 deletions(-) diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index 69efc70a760..b5949a82cd1 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -36,7 +36,7 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id) void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) { - std::unique_lock lock(this->mu); + std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if unlikely (iter == this->ctes.end()) // Maybe the task is cancelled and the cte has been released @@ -59,7 +59,7 @@ void CTEManager::releaseCTE(const String & query_id_and_cte_id) this->ctes.erase(iter); } -std::shared_ptr CTEManager::getCTEImpl( +std::shared_ptr CTEManager::getCTE( const String & query_id_and_cte_id, Int32 concurrency, Int32 expected_sink_num, diff --git a/dbms/src/Flash/Mpp/CTEManager.h b/dbms/src/Flash/Mpp/CTEManager.h index 3a13c2eb08d..a0719980a36 100644 --- a/dbms/src/Flash/Mpp/CTEManager.h +++ b/dbms/src/Flash/Mpp/CTEManager.h @@ -52,37 +52,19 @@ class CTEWithCounter Int32 expected_source_num; }; -// TODO Test this class with UT class CTEManager { public: - std::shared_ptr getCTEBySink( + std::shared_ptr getCTE( const String & query_id_and_cte_id, Int32 concurrency, Int32 expected_sink_num, - Int32 expected_source_num) - { - return this->getCTEImpl(query_id_and_cte_id, concurrency, expected_sink_num, expected_source_num); - } - std::shared_ptr getCTEBySource( - const String & query_id_and_cte_id, - Int32 concurrency, - Int32 expected_sink_num, - Int32 expected_source_num) - { - return this->getCTEImpl(query_id_and_cte_id, concurrency, expected_sink_num, expected_source_num); - } + Int32 expected_source_num); void releaseCTEBySource(const String & query_id_and_cte_id); void releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id); void releaseCTE(const String & query_id_and_cte_id); private: - std::shared_ptr getCTEImpl( - const String & query_id_and_cte_id, - Int32 concurrency, - Int32 expected_sink_num, - Int32 expected_source_num); - std::mutex mu; std::unordered_map ctes; }; diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 72e0c5d5f67..b95bd1cdb2c 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -153,7 +153,6 @@ MPPTask::~MPPTask() if (query_memory_tracker != nullptr && current_memory_tracker != query_memory_tracker) current_memory_tracker = query_memory_tracker; abortTunnels("", true); - abortCTE(""); LOG_INFO(log, "finish MPPTask: {}, total run time is {} ms", id.toString(), total_run_time_ms); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index a7931089546..28f938ab54a 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -58,7 +58,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( if (fine_grained_shuffle.enabled()) concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); - std::shared_ptr cte = context.getCTEManager()->getCTEBySink( + std::shared_ptr cte = context.getCTEManager()->getCTE( query_id_and_cte_id, concurrency, this->expected_sink_num, diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 1b51e959e28..3bc4cd25440 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -59,11 +59,9 @@ void CTE::registerTask(size_t partition_id, TaskPtr && task, NotifyType type) void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t partition_id) { - CTEOpStatus status; - std::shared_lock rw_lock(this->rw_lock); std::lock_guard lock(*this->partitions[partition_id].mu); - status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); + CTEOpStatus status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); if (status == CTEOpStatus::BLOCK_NOT_AVAILABLE) { diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 4f6ff82da45..dd59cf53ddf 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -36,8 +36,7 @@ class CTEReader Int32 expected_source_num_) : query_id_and_cte_id(query_id_and_cte_id_) , cte_manager(cte_manager_) - , cte(cte_manager_ - ->getCTEBySource(query_id_and_cte_id_, partition_num, expected_sink_num_, expected_source_num_)) + , cte(cte_manager_->getCTE(query_id_and_cte_id_, partition_num, expected_sink_num_, expected_source_num_)) , cte_reader_id(this->cte->getCTEReaderID()) {} From 5c3918df6ebcd86b02fd7b18342e2e339cbb381a Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 2 Jul 2025 19:41:23 +0800 Subject: [PATCH 080/118] save --- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 13 ++++++++++++ .../Flash/Planner/Plans/PhysicalCTESource.cpp | 1 - dbms/src/Interpreters/CTESpillContext.cpp | 15 ++++++++++++-- dbms/src/Interpreters/CTESpillContext.h | 9 +++++++-- dbms/src/Operators/CTE.cpp | 20 +++++++++++++++++++ dbms/src/Operators/CTE.h | 11 +++++++--- dbms/src/Operators/CTEPartition.h | 14 +++++++++---- 7 files changed, 71 insertions(+), 12 deletions(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 28f938ab54a..7c489417928 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -69,6 +69,19 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( builder.setSinkOp(std::make_unique(exec_context, log->identifier(), cte, id)); id++; }); + + const Settings & settings = context.getSettingsRef(); + SpillConfig spill_config( + context.getTemporaryPath(), + "", + settings.max_cached_data_bytes_in_spiller, + settings.max_spilled_rows_per_file, + settings.max_spilled_bytes_per_file, + context.getFileProvider(), + settings.max_threads, + settings.max_block_size); + + cte->initCTESpillContext(spill_config, group_builder.getCurrentHeader(), 0, query_id_and_cte_id); } void PhysicalCTESink::finalizeImpl(const Names & parent_require) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index db01550f8b8..c2657cdfd51 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -79,7 +79,6 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( std::make_unique(exec_context, log->identifier(), cte_reader, i, schema)); } - context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } diff --git a/dbms/src/Interpreters/CTESpillContext.cpp b/dbms/src/Interpreters/CTESpillContext.cpp index 43eb4aec02f..fe74d5400e8 100644 --- a/dbms/src/Interpreters/CTESpillContext.cpp +++ b/dbms/src/Interpreters/CTESpillContext.cpp @@ -30,7 +30,18 @@ SpillerSharedPtr CTESpillContext::getSpillAt(size_t idx) if (idx < spiller_num) return this->spillers[idx]; - // TODO create new spiller - return this->spillers[idx]; + SpillConfig config( + this->spill_config.spill_dir, + fmt::format("cte_spill_{}", idx), + this->spill_config.max_cached_data_bytes_in_spiller, + this->spill_config.max_spilled_rows_per_file, + this->spill_config.max_spilled_bytes_per_file, + this->spill_config.file_provider, + this->spill_config.for_all_constant_max_streams, + this->spill_config.for_all_constant_block_size); + + this->spillers.push_back( + std::make_shared(config, false, this->partition_num, this->spill_block_schema, this->log)); + return this->spillers.back(); } } // namespace DB diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h index dced5223a86..e47b68ea9a1 100644 --- a/dbms/src/Interpreters/CTESpillContext.h +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -26,13 +26,16 @@ class CTESpillContext final : public OperatorSpillContext { public: CTESpillContext( + size_t partition_num_, const SpillConfig & spill_config_, const Block & spill_block_schema_, UInt64 operator_spill_threshold_, - const LoggerPtr & log_) - : OperatorSpillContext(operator_spill_threshold_, "cte", log_) + const String & query_id_and_cte_id_) + : OperatorSpillContext(operator_spill_threshold_, "cte", Logger::get(query_id_and_cte_id_)) + , partition_num(partition_num_) , spill_config(spill_config_) , spill_block_schema(spill_block_schema_) + , query_id_and_cte_id(query_id_and_cte_id_) {} ~CTESpillContext() override = default; @@ -46,9 +49,11 @@ class CTESpillContext final : public OperatorSpillContext private: std::mutex mu; + size_t partition_num; SpillConfig spill_config; Block spill_block_schema; std::vector spillers; + String query_id_and_cte_id; }; } // namespace DB diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index de8b402cb1a..562710ff9e9 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -13,17 +13,37 @@ // limitations under the License. #include +#include #include #include +#include #include #include #include +#include #include #include namespace DB { +void CTE::initCTESpillContext( + const SpillConfig & spill_config_, + const Block & spill_block_schema_, + UInt64 operator_spill_threshold_, + const String & query_id_and_cte_id) +{ + std::unique_lock lock(this->rw_lock); + this->cte_spill_context = std::make_shared( + this->partition_num, + spill_config_, + spill_block_schema_, + operator_spill_threshold_, + query_id_and_cte_id); + for (auto & item : this->partitions) + item.init(this->cte_spill_context); +} + CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block & block) { std::shared_lock rw_lock(this->rw_lock); diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index ff286eef47d..88c434d1f9f 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -35,9 +35,15 @@ class CTE : partition_num(partition_num_) { for (size_t i = 0; i < this->partition_num; i++) - this->partitions.push_back(CTEPartition(i, &(this->cte_spill_context))); + this->partitions.push_back(CTEPartition(i)); } + void initCTESpillContext( + const SpillConfig & spill_config_, + const Block & spill_block_schema_, + UInt64 operator_spill_threshold_, + const String & query_id_and_cte_id); + void checkPartitionNum(size_t partition_num) const { RUNTIME_CHECK_MSG( @@ -123,8 +129,6 @@ class CTE size_t partition_num; std::vector partitions; - CTESpillContext cte_spill_context; // TODO initialize it - std::shared_mutex rw_lock; bool is_eof = false; bool is_cancelled = false; @@ -133,6 +137,7 @@ class CTE tipb::SelectResponse resp; String err_msg; + std::shared_ptr cte_spill_context; }; class CTEIONotifier : public NotifyFuture diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 70ed73795e3..439b11052bd 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -44,14 +44,15 @@ enum class CTEOpStatus struct CTEPartition { - CTEPartition(size_t partition_id_, CTESpillContext * spill_context_) + explicit CTEPartition(size_t partition_id_) : partition_id(partition_id_) , mu(std::make_unique()) , pipe_cv(std::make_unique()) , status_lock(std::make_unique()) - , spill_context(spill_context_) {} + void init(std::shared_ptr spill_context_) { this->spill_context = spill_context_; } + size_t getIdxInMemoryNoLock(size_t cte_reader_id); bool isBlockAvailableInDiskNoLock(size_t cte_reader_id) { @@ -61,10 +62,15 @@ struct CTEPartition { return this->getIdxInMemoryNoLock(cte_reader_id) < this->blocks.size(); } - bool exceedMemoryThresholdNoLock() const { return this->memory_usage >= this->memory_threoshold; } void setCTEPartitionStatusNoLock(CTEPartitionStatus status) { this->status = status; } bool isSpillTriggeredNoLock() const { return this->total_block_in_disk_num > 0; } void addIdxNoLock(size_t cte_reader_id) { this->fetch_block_idxs[cte_reader_id]++; } + bool exceedMemoryThresholdNoLock() const + { + if (this->memory_threoshold == 0) + return false; + return this->memory_usage >= this->memory_threoshold; + } CTEOpStatus pushBlock(const Block & block); CTEOpStatus tryGetBlockAt(size_t cte_reader_id, Block & block); @@ -98,6 +104,6 @@ struct CTEPartition std::unordered_map cte_reader_restore_streams; UInt64 total_block_in_disk_num = 0; - CTESpillContext * spill_context; + std::shared_ptr spill_context; }; } // namespace DB From 1501ac636c6d6d9e974c73328dd43dccd6e75ea9 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 3 Jul 2025 13:55:05 +0800 Subject: [PATCH 081/118] address comments --- dbms/src/Flash/Coprocessor/DAGContext.h | 1 - dbms/src/Flash/Executor/PipelineExecutorContext.h | 1 - dbms/src/Flash/Mpp/CTEManager.cpp | 4 +--- dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp | 5 ++--- dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp | 3 --- 5 files changed, 3 insertions(+), 11 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 76fa708d629..2a2526d311b 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -371,7 +371,6 @@ class DAGContext std::vector> getCTEs() const { return this->ctes; } void addCTE(std::shared_ptr & cte) { this->ctes.push_back(cte); } - bool hasCTESource() const { return this->has_cte_source; } void setHasCTESource() { this->has_cte_source = true; } public: diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.h b/dbms/src/Flash/Executor/PipelineExecutorContext.h index a85d921cc67..70d621d50cf 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.h +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.h @@ -156,7 +156,6 @@ class PipelineExecutorContext : private boost::noncopyable } void addCTE(std::shared_ptr cte) { this->dag_context->addCTE(cte); } - void addCTE(std::shared_ptr & cte) { this->dag_context->addCTE(cte); } void setHasCTESource() { this->dag_context->setHasCTESource(); } diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index b5949a82cd1..2c210dd642d 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -54,9 +54,7 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin void CTEManager::releaseCTE(const String & query_id_and_cte_id) { std::lock_guard lock(this->mu); - auto iter = this->ctes.find(query_id_and_cte_id); - if (iter != this->ctes.end()) - this->ctes.erase(iter); + this->ctes.erase(this->ctes.find(query_id_and_cte_id)); } std::shared_ptr CTEManager::getCTE( diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 28f938ab54a..87db4603012 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -55,15 +55,14 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); exec_context.setQueryIDAndCTEID(query_id_and_cte_id); - if (fine_grained_shuffle.enabled()) - concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); - std::shared_ptr cte = context.getCTEManager()->getCTE( query_id_and_cte_id, concurrency, this->expected_sink_num, this->expected_source_num); + RUNTIME_CHECK(group_builder.concurrency() == concurrency); + size_t id = 0; group_builder.transform([&](auto & builder) { builder.setSinkOp(std::make_unique(exec_context, log->identifier(), cte, id)); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index db01550f8b8..2c94624beca 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -59,9 +59,6 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( Context & context, size_t concurrency) { - if (fine_grained_shuffle.enabled()) - concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); - String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); exec_context.setQueryIDAndCTEID(query_id_and_cte_id); exec_context.setHasCTESource(); From 288b679f0afcea14e3eca25369204cabdd860ae3 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 3 Jul 2025 15:51:07 +0800 Subject: [PATCH 082/118] save --- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 7 +++++-- dbms/src/Interpreters/CTESpillContext.h | 19 ++++++++----------- dbms/src/Interpreters/Settings.h | 1 + dbms/src/Operators/CTE.cpp | 3 ++- dbms/src/Operators/CTE.h | 1 + dbms/src/Operators/CTEPartition.h | 8 ++++++-- 6 files changed, 23 insertions(+), 16 deletions(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 7c557a9ec7a..1e5d325665e 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -20,7 +20,6 @@ #include #include -#include namespace DB { @@ -80,7 +79,11 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( settings.max_threads, settings.max_block_size); - cte->initCTESpillContext(spill_config, group_builder.getCurrentHeader(), 0, query_id_and_cte_id); + cte->initCTESpillContext( + spill_config, + group_builder.getCurrentHeader(), + settings.max_bytes_before_cte_spill, + query_id_and_cte_id); } void PhysicalCTESink::finalizeImpl(const Names & parent_require) diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h index e47b68ea9a1..4a414e179d1 100644 --- a/dbms/src/Interpreters/CTESpillContext.h +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include @@ -22,29 +23,23 @@ namespace DB { -class CTESpillContext final : public OperatorSpillContext +class CTESpillContext { public: CTESpillContext( size_t partition_num_, const SpillConfig & spill_config_, const Block & spill_block_schema_, - UInt64 operator_spill_threshold_, + UInt64 spill_threshold_, const String & query_id_and_cte_id_) - : OperatorSpillContext(operator_spill_threshold_, "cte", Logger::get(query_id_and_cte_id_)) - , partition_num(partition_num_) + : partition_num(partition_num_) , spill_config(spill_config_) , spill_block_schema(spill_block_schema_) + , spill_threshold(spill_threshold_) , query_id_and_cte_id(query_id_and_cte_id_) + , log(Logger::get(query_id_and_cte_id_)) {} - ~CTESpillContext() override = default; - - Int64 getTotalRevocableMemoryImpl() override { throw Exception(""); } // TODO implement - bool supportFurtherSpill() const override { throw Exception(""); } // TODO implement - bool supportAutoTriggerSpill() const override { throw Exception(""); } // TODO implement - Int64 triggerSpillImpl(Int64) override { throw Exception(""); } // TODO implement - SpillerSharedPtr getSpillAt(size_t idx); private: @@ -52,8 +47,10 @@ class CTESpillContext final : public OperatorSpillContext size_t partition_num; SpillConfig spill_config; Block spill_block_schema; + UInt64 spill_threshold; std::vector spillers; String query_id_and_cte_id; + LoggerPtr log; }; } // namespace DB diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index c18ba194308..39f88275970 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -313,6 +313,7 @@ struct Settings M(SettingUInt64, async_cqs, 1, "grpc async cqs") \ M(SettingUInt64, preallocated_request_count_per_poller, 20, "grpc preallocated_request_count_per_poller") \ M(SettingUInt64, max_bytes_before_external_join, 0, "max bytes used by join before spill, 0 as the default value, 0 means no limit") \ + M(SettingUInt64, max_bytes_before_cte_spill, 0, "max bytes used by cte before spill, 0 as the default value, 0 means no limit") \ M(SettingInt64, join_restore_concurrency, 0, "join restore concurrency, negative value means restore join serially, 0 means TiFlash choose restore concurrency automatically, 0 as the default value") \ M(SettingUInt64, max_cached_data_bytes_in_spiller, 1024ULL * 1024 * 20, "Max cached data bytes in spiller before spilling, 20 MB as the default value, 0 means no limit") \ M(SettingUInt64, max_spilled_rows_per_file, 200000, "Max spilled data rows per spill file, 200000 as the default value, 0 means no limit.") \ diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 562710ff9e9..0fcd1e27de3 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -40,8 +40,9 @@ void CTE::initCTESpillContext( spill_block_schema_, operator_spill_threshold_, query_id_and_cte_id); + size_t memory_threoshold = operator_spill_threshold_ / this->partition_num; for (auto & item : this->partitions) - item.init(this->cte_spill_context); + item.init(this->cte_spill_context, memory_threoshold); } CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block & block) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 88c434d1f9f..6e7a94dce09 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -34,6 +34,7 @@ class CTE explicit CTE(size_t partition_num_) : partition_num(partition_num_) { + RUNTIME_CHECK(this->partition_num > 0); for (size_t i = 0; i < this->partition_num; i++) this->partitions.push_back(CTEPartition(i)); } diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 439b11052bd..2dfc83d582b 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -51,7 +51,11 @@ struct CTEPartition , status_lock(std::make_unique()) {} - void init(std::shared_ptr spill_context_) { this->spill_context = spill_context_; } + void init(std::shared_ptr spill_context_, size_t memory_threoshold_) + { + this->spill_context = spill_context_; + this->memory_threoshold = memory_threoshold_; + } size_t getIdxInMemoryNoLock(size_t cte_reader_id); bool isBlockAvailableInDiskNoLock(size_t cte_reader_id) @@ -91,7 +95,7 @@ struct CTEPartition Blocks blocks; std::unordered_map fetch_block_idxs; size_t memory_usage = 0; - size_t memory_threoshold = 0; // TODO initialize it + size_t memory_threoshold = 0; std::unique_ptr pipe_cv; Blocks tmp_blocks; From bfd65c53a3de8c08e82c9a5c78626dd652e30c24 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 4 Jul 2025 12:00:18 +0800 Subject: [PATCH 083/118] tweaking --- dbms/src/DataStreams/NativeBlockInputStream.h | 6 ---- dbms/src/IO/Buffer/ReadBuffer.h | 3 -- dbms/src/Interpreters/CTESpillContext.h | 3 -- dbms/src/Operators/CTE.cpp | 7 ++--- dbms/src/Operators/CTEPartition.cpp | 29 +++++++++---------- dbms/src/Operators/CTEPartition.h | 12 ++++---- dbms/src/Operators/CTESinkOp.cpp | 5 ++-- 7 files changed, 25 insertions(+), 40 deletions(-) diff --git a/dbms/src/DataStreams/NativeBlockInputStream.h b/dbms/src/DataStreams/NativeBlockInputStream.h index f5bc84cb39a..c08467a8a5a 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.h +++ b/dbms/src/DataStreams/NativeBlockInputStream.h @@ -101,12 +101,6 @@ class NativeBlockInputStream : public IProfilingBlockInputStream Block getHeader() const override; - void seek(Int64 pos) - { - this->istr.seek(pos); - this->istr.next(); // Trigger update for read buffer - } - protected: Block readImpl() override; diff --git a/dbms/src/IO/Buffer/ReadBuffer.h b/dbms/src/IO/Buffer/ReadBuffer.h index 056b1a999a8..921d15eb90b 100644 --- a/dbms/src/IO/Buffer/ReadBuffer.h +++ b/dbms/src/IO/Buffer/ReadBuffer.h @@ -175,9 +175,6 @@ class ReadBuffer : public BufferBase */ virtual size_t readBig(char * to, size_t n) { return read(to, n); } - // Set the read position for file - virtual void seek(Int64) { throw Exception("Not implemented yet"); } - protected: /// The number of bytes to ignore from the initial position of `working_buffer` buffer. size_t working_buffer_offset = 0; diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h index 4a414e179d1..14491926b2d 100644 --- a/dbms/src/Interpreters/CTESpillContext.h +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -30,12 +30,10 @@ class CTESpillContext size_t partition_num_, const SpillConfig & spill_config_, const Block & spill_block_schema_, - UInt64 spill_threshold_, const String & query_id_and_cte_id_) : partition_num(partition_num_) , spill_config(spill_config_) , spill_block_schema(spill_block_schema_) - , spill_threshold(spill_threshold_) , query_id_and_cte_id(query_id_and_cte_id_) , log(Logger::get(query_id_and_cte_id_)) {} @@ -47,7 +45,6 @@ class CTESpillContext size_t partition_num; SpillConfig spill_config; Block spill_block_schema; - UInt64 spill_threshold; std::vector spillers; String query_id_and_cte_id; diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 0fcd1e27de3..4fccefb6897 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -38,7 +38,6 @@ void CTE::initCTESpillContext( this->partition_num, spill_config_, spill_block_schema_, - operator_spill_threshold_, query_id_and_cte_id); size_t memory_threoshold = operator_spill_threshold_ / this->partition_num; for (auto & item : this->partitions) @@ -51,7 +50,7 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block if unlikely (this->is_cancelled) return CTEOpStatus::CANCELLED; - auto status = this->partitions[partition_id].tryGetBlockAt(cte_reader_id, block); + auto status = this->partitions[partition_id].tryGetBlock(cte_reader_id, block); switch (status) { case CTEOpStatus::BLOCK_NOT_AVAILABLE: @@ -104,7 +103,7 @@ void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_ return; } - std::lock_guard status_lock(*(this->partitions[partition_id].status_lock)); + std::lock_guard aux_lock(*(this->partitions[partition_id].aux_lock)); if (this->partitions[partition_id].status == CTEPartitionStatus::IN_SPILLING) { this->notifyTaskDirectly(partition_id, std::move(task)); @@ -127,7 +126,7 @@ void CTE::checkInSpillingAndRegisterTask(TaskPtr && task, size_t partition_id) return; } - std::lock_guard status_lock(*(this->partitions[partition_id].status_lock)); + std::lock_guard aux_lock(*(this->partitions[partition_id].aux_lock)); if (this->partitions[partition_id].status == CTEPartitionStatus::IN_SPILLING) this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); else diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 4a1d085dd71..150527e1b1e 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -28,9 +28,9 @@ size_t CTEPartition::getIdxInMemoryNoLock(size_t cte_reader_id) return this->fetch_block_idxs[cte_reader_id] - this->total_block_in_disk_num; } -CTEOpStatus CTEPartition::tryGetBlockAt(size_t cte_reader_id, Block & block) +CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) { - std::lock_guard status_lock(*(this->status_lock)); + std::lock_guard aux_lock(*(this->aux_lock)); if (this->status == CTEPartitionStatus::IN_SPILLING) return CTEOpStatus::IO_OUT; @@ -49,20 +49,18 @@ CTEOpStatus CTEPartition::tryGetBlockAt(size_t cte_reader_id, Block & block) CTEOpStatus CTEPartition::pushBlock(const Block & block) { - std::unique_lock status_lock(*(this->status_lock)); + std::unique_lock aux_lock(*(this->aux_lock)); if (this->status != CTEPartitionStatus::NORMAL) { if likely (block.rows() != 0) { - std::lock_guard lock(*this->mu); - // Block memory usage will be calculated after the finish of spill this->tmp_blocks.push_back(block); } return CTEOpStatus::IO_OUT; } - // mu must be held after status_lock so that we will not be blocked when spill is triggered. + // mu must be held after aux_lock so that we will not be blocked by spill. // Blocked in cpu pool is very bad. std::lock_guard lock(*this->mu); @@ -82,7 +80,7 @@ CTEOpStatus CTEPartition::spillBlocks() { std::unique_lock lock(*(this->mu), std::defer_lock); { - std::lock_guard status_lock(*(this->status_lock)); + std::lock_guard aux_lock(*(this->aux_lock)); switch (this->status) { case CTEPartitionStatus::NORMAL: @@ -95,12 +93,11 @@ CTEOpStatus CTEPartition::spillBlocks() } lock.lock(); + for (const auto & block : this->tmp_blocks) + this->blocks.push_back(block); + this->tmp_blocks.clear(); } - for (const auto & block : this->tmp_blocks) - this->blocks.push_back(block); - this->tmp_blocks.clear(); - auto cte_reader_num = this->fetch_block_idxs.size(); std::vector split_idxs{0}; split_idxs.reserve(cte_reader_num + 1); @@ -129,7 +126,7 @@ CTEOpStatus CTEPartition::spillBlocks() this->blocks.clear(); this->memory_usage = 0; - std::lock_guard status_lock(*(this->status_lock)); + std::lock_guard aux_lock(*(this->aux_lock)); this->setCTEPartitionStatusNoLock(CTEPartitionStatus::NORMAL); // Many tasks may be waiting for the finish of spill @@ -141,7 +138,7 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) { std::unique_lock lock(*(this->mu), std::defer_lock); { - std::lock_guard status_lock(*(this->status_lock)); + std::lock_guard aux_lock(*(this->aux_lock)); switch (this->status) { case CTEPartitionStatus::IN_SPILLING: @@ -162,8 +159,10 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) auto [iter, _] = this->cte_reader_restore_streams.insert(std::make_pair(cte_reader_id, nullptr)); if (iter->second == nullptr) { - auto spiller_iter = this->spillers.find(cte_reader_id); - RUNTIME_CHECK_MSG(spiller_iter == this->spillers.end(), "cte reader {} can't find spiller", cte_reader_id); + auto spiller_iter = this->spillers.find(this->fetch_block_idxs[cte_reader_id]); + if (spiller_iter == this->spillers.end()) + // All blocks in disk have been consumed + return CTEOpStatus::OK; auto streams = spiller_iter->second->restoreBlocks(this->partition_id, 1); RUNTIME_CHECK(streams.size() == 1); iter->second = streams[0]; diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 2dfc83d582b..cf899988576 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -48,7 +48,7 @@ struct CTEPartition : partition_id(partition_id_) , mu(std::make_unique()) , pipe_cv(std::make_unique()) - , status_lock(std::make_unique()) + , aux_lock(std::make_unique()) {} void init(std::shared_ptr spill_context_, size_t memory_threoshold_) @@ -73,11 +73,11 @@ struct CTEPartition { if (this->memory_threoshold == 0) return false; - return this->memory_usage >= this->memory_threoshold; + return this->memory_usage > this->memory_threoshold; } CTEOpStatus pushBlock(const Block & block); - CTEOpStatus tryGetBlockAt(size_t cte_reader_id, Block & block); + CTEOpStatus tryGetBlock(size_t cte_reader_id, Block & block); CTEOpStatus spillBlocks(); CTEOpStatus getBlockFromDisk(size_t cte_reader_id, Block & block); @@ -97,11 +97,11 @@ struct CTEPartition size_t memory_usage = 0; size_t memory_threoshold = 0; std::unique_ptr pipe_cv; - Blocks tmp_blocks; - // Protecting cte_status - std::unique_ptr status_lock; + // Protecting cte_status and tmp_blocks + std::unique_ptr aux_lock; CTEPartitionStatus status = CTEPartitionStatus::NORMAL; + Blocks tmp_blocks; std::vector block_in_disk_nums; std::unordered_map spillers; diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index e2c9ed70986..e71acf161e0 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -13,12 +13,12 @@ // limitations under the License. #include +#include #include #include #include -#include "Operators/CTEPartition.h" -#include "magic_enum.hpp" +#include namespace DB { @@ -59,7 +59,6 @@ OperatorStatus CTESinkOp::executeIOImpl() case CTEOpStatus::IO_OUT: // CTE is spilling blocks to disk, we need to wait the finish of spill DB::setNotifyFuture(&(this->io_notifier)); - return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::CANCELLED: return OperatorStatus::CANCELLED; From 6f09386b7b8a2c5981129a3408966fd8a8446087 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 4 Jul 2025 20:42:18 +0800 Subject: [PATCH 084/118] fix bugs --- dbms/src/Flash/Coprocessor/DAGContext.h | 2 -- dbms/src/Flash/Executor/PipelineExecutorContext.h | 2 -- dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp | 4 ++-- dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp | 3 ++- dbms/src/Operators/CTEPartition.cpp | 1 + 5 files changed, 5 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 2a2526d311b..eef593b116e 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -371,7 +371,6 @@ class DAGContext std::vector> getCTEs() const { return this->ctes; } void addCTE(std::shared_ptr & cte) { this->ctes.push_back(cte); } - void setHasCTESource() { this->has_cte_source = true; } public: DAGRequest dag_request; @@ -491,7 +490,6 @@ class DAGContext String connection_alias; String query_id_and_cte_id; - bool has_cte_source = false; std::vector> ctes; }; diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.h b/dbms/src/Flash/Executor/PipelineExecutorContext.h index 70d621d50cf..0f39cfd5803 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.h +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.h @@ -157,8 +157,6 @@ class PipelineExecutorContext : private boost::noncopyable void addCTE(std::shared_ptr cte) { this->dag_context->addCTE(cte); } - void setHasCTESource() { this->dag_context->setHasCTESource(); } - private: bool setExceptionPtr(const std::exception_ptr & exception_ptr_); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 1e5d325665e..1d6adefdb0c 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -60,7 +60,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( this->expected_sink_num, this->expected_source_num); - RUNTIME_CHECK(group_builder.concurrency() == concurrency); + RUNTIME_CHECK(group_builder.concurrency() <= concurrency); size_t id = 0; group_builder.transform([&](auto & builder) { @@ -71,7 +71,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( const Settings & settings = context.getSettingsRef(); SpillConfig spill_config( context.getTemporaryPath(), - "", + " ", settings.max_cached_data_bytes_in_spiller, settings.max_spilled_rows_per_file, settings.max_spilled_bytes_per_file, diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index b469811acf7..f9c8cac9677 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -61,7 +61,8 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( { String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); exec_context.setQueryIDAndCTEID(query_id_and_cte_id); - exec_context.setHasCTESource(); + + RUNTIME_CHECK(group_builder.concurrency() <= concurrency); auto cte_reader = std::make_shared( query_id_and_cte_id, diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 150527e1b1e..223e53b1c05 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -44,6 +44,7 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) auto idx = this->getIdxInMemoryNoLock(cte_reader_id); block = this->blocks[idx]; + this->addIdxNoLock(cte_reader_id); return CTEOpStatus::OK; } From 7f79f2073443185d55caf00e967cebdb565daa62 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 4 Jul 2025 20:44:03 +0800 Subject: [PATCH 085/118] address comments --- dbms/src/Flash/Coprocessor/DAGContext.h | 2 -- dbms/src/Flash/Executor/PipelineExecutorContext.h | 2 -- dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp | 2 +- dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp | 4 ++-- 4 files changed, 3 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 2a2526d311b..eef593b116e 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -371,7 +371,6 @@ class DAGContext std::vector> getCTEs() const { return this->ctes; } void addCTE(std::shared_ptr & cte) { this->ctes.push_back(cte); } - void setHasCTESource() { this->has_cte_source = true; } public: DAGRequest dag_request; @@ -491,7 +490,6 @@ class DAGContext String connection_alias; String query_id_and_cte_id; - bool has_cte_source = false; std::vector> ctes; }; diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.h b/dbms/src/Flash/Executor/PipelineExecutorContext.h index 70d621d50cf..0f39cfd5803 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.h +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.h @@ -157,8 +157,6 @@ class PipelineExecutorContext : private boost::noncopyable void addCTE(std::shared_ptr cte) { this->dag_context->addCTE(cte); } - void setHasCTESource() { this->dag_context->setHasCTESource(); } - private: bool setExceptionPtr(const std::exception_ptr & exception_ptr_); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 87db4603012..5e994229c76 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -61,7 +61,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( this->expected_sink_num, this->expected_source_num); - RUNTIME_CHECK(group_builder.concurrency() == concurrency); + RUNTIME_CHECK(group_builder.concurrency() <= concurrency); size_t id = 0; group_builder.transform([&](auto & builder) { diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index 2c94624beca..f9c8cac9677 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -61,7 +61,8 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( { String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); exec_context.setQueryIDAndCTEID(query_id_and_cte_id); - exec_context.setHasCTESource(); + + RUNTIME_CHECK(group_builder.concurrency() <= concurrency); auto cte_reader = std::make_shared( query_id_and_cte_id, @@ -76,7 +77,6 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( std::make_unique(exec_context, log->identifier(), cte_reader, i, schema)); } - context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } From aa27b1f4cfee4bdad98cbbc3b085dd2d002ab056 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 7 Jul 2025 11:16:49 +0800 Subject: [PATCH 086/118] add log --- dbms/src/Interpreters/CTESpillContext.h | 2 ++ dbms/src/Operators/CTEPartition.cpp | 17 +++++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h index 14491926b2d..39f98a5353f 100644 --- a/dbms/src/Interpreters/CTESpillContext.h +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -39,6 +39,8 @@ class CTESpillContext {} SpillerSharedPtr getSpillAt(size_t idx); + LoggerPtr getLog() const { return this->log; } + String getQueryIdAndCTEId() const { return this->query_id_and_cte_id; } private: std::mutex mu; diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 223e53b1c05..54bb67fd42d 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -79,6 +79,12 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) CTEOpStatus CTEPartition::spillBlocks() { + LOG_INFO( + this->spill_context->getLog(), + fmt::format( + "Partition {} starts cte spill for {}", + this->partition_id, + this->spill_context->getQueryIdAndCTEId())); std::unique_lock lock(*(this->mu), std::defer_lock); { std::lock_guard aux_lock(*(this->aux_lock)); @@ -95,7 +101,10 @@ CTEOpStatus CTEPartition::spillBlocks() lock.lock(); for (const auto & block : this->tmp_blocks) + { + this->memory_usage += block.bytes(); this->blocks.push_back(block); + } this->tmp_blocks.clear(); } @@ -124,6 +133,14 @@ CTEOpStatus CTEPartition::spillBlocks() spiller->spillBlocks(std::move(spilled_blocks), this->partition_id); } + LOG_INFO( + this->spill_context->getLog(), + fmt::format( + "Partition {} finishes cte spill for {}, spilled memory: {}", + this->partition_id, + this->spill_context->getQueryIdAndCTEId(), + this->memory_usage)); + this->blocks.clear(); this->memory_usage = 0; From 5d53c4c70f84bb15284a13ce4065c2bbe6b311a9 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 7 Jul 2025 17:58:59 +0800 Subject: [PATCH 087/118] fix --- dbms/src/Flash/Mpp/CTEManager.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index 2c210dd642d..b5949a82cd1 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -54,7 +54,9 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin void CTEManager::releaseCTE(const String & query_id_and_cte_id) { std::lock_guard lock(this->mu); - this->ctes.erase(this->ctes.find(query_id_and_cte_id)); + auto iter = this->ctes.find(query_id_and_cte_id); + if (iter != this->ctes.end()) + this->ctes.erase(iter); } std::shared_ptr CTEManager::getCTE( From 209e9ada7b5e47a2bd412e9f160ed8cdd924a694 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 8 Jul 2025 18:12:22 +0800 Subject: [PATCH 088/118] fix bugs, more bugs --- dbms/src/Core/Spiller.h | 1 - dbms/src/Interpreters/CTESpillContext.cpp | 17 +--- dbms/src/Interpreters/CTESpillContext.h | 4 +- dbms/src/Operators/CTE.h | 6 ++ dbms/src/Operators/CTEPartition.cpp | 94 +++++++++++++++-------- dbms/src/Operators/CTEPartition.h | 24 +++++- dbms/src/Operators/CTEReader.cpp | 5 +- dbms/src/Operators/CTESinkOp.cpp | 12 ++- dbms/src/Operators/CTESourceOp.cpp | 13 +++- 9 files changed, 119 insertions(+), 57 deletions(-) diff --git a/dbms/src/Core/Spiller.h b/dbms/src/Core/Spiller.h index 7eb43354b61..f5bac67e3fc 100644 --- a/dbms/src/Core/Spiller.h +++ b/dbms/src/Core/Spiller.h @@ -166,6 +166,5 @@ class Spiller }; using SpillerPtr = std::unique_ptr; -using SpillerSharedPtr = std::shared_ptr; } // namespace DB diff --git a/dbms/src/Interpreters/CTESpillContext.cpp b/dbms/src/Interpreters/CTESpillContext.cpp index fe74d5400e8..9740be358ce 100644 --- a/dbms/src/Interpreters/CTESpillContext.cpp +++ b/dbms/src/Interpreters/CTESpillContext.cpp @@ -19,20 +19,11 @@ namespace DB { -SpillerSharedPtr CTESpillContext::getSpillAt(size_t idx) +SpillerPtr CTESpillContext::getSpiller(size_t partition_id, size_t spill_id) { - std::lock_guard lock(this->mu); - auto spiller_num = this->spillers.size(); - - // The spiller whose idx is lower that the parameter idx must have been created before - RUNTIME_CHECK_MSG(idx <= spiller_num, "idx: {}, spiller_num: {}", idx, spiller_num); - - if (idx < spiller_num) - return this->spillers[idx]; - SpillConfig config( this->spill_config.spill_dir, - fmt::format("cte_spill_{}", idx), + fmt::format("cte_spill_{}_{}", partition_id, spill_id), this->spill_config.max_cached_data_bytes_in_spiller, this->spill_config.max_spilled_rows_per_file, this->spill_config.max_spilled_bytes_per_file, @@ -40,8 +31,6 @@ SpillerSharedPtr CTESpillContext::getSpillAt(size_t idx) this->spill_config.for_all_constant_max_streams, this->spill_config.for_all_constant_block_size); - this->spillers.push_back( - std::make_shared(config, false, this->partition_num, this->spill_block_schema, this->log)); - return this->spillers.back(); + return std::make_unique(config, false, this->partition_num, this->spill_block_schema, this->log); } } // namespace DB diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h index 39f98a5353f..ca8436420a2 100644 --- a/dbms/src/Interpreters/CTESpillContext.h +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -38,17 +38,15 @@ class CTESpillContext , log(Logger::get(query_id_and_cte_id_)) {} - SpillerSharedPtr getSpillAt(size_t idx); + SpillerPtr getSpiller(size_t partition_id, size_t spill_id); LoggerPtr getLog() const { return this->log; } String getQueryIdAndCTEId() const { return this->query_id_and_cte_id; } private: - std::mutex mu; size_t partition_num; SpillConfig spill_config; Block spill_block_schema; - std::vector spillers; String query_id_and_cte_id; LoggerPtr log; }; diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 6e7a94dce09..72b567871b3 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -39,6 +39,12 @@ class CTE this->partitions.push_back(CTEPartition(i)); } + ~CTE() + { + for (auto & p : this->partitions) + p.debugOutput(); + } + void initCTESpillContext( const SpillConfig & spill_config_, const Block & spill_block_schema_, diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 54bb67fd42d..7b094d3dbed 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -32,7 +33,7 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) { std::lock_guard aux_lock(*(this->aux_lock)); if (this->status == CTEPartitionStatus::IN_SPILLING) - return CTEOpStatus::IO_OUT; + return CTEOpStatus::WAIT_SPILL; std::lock_guard lock(*this->mu); @@ -45,26 +46,35 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) auto idx = this->getIdxInMemoryNoLock(cte_reader_id); block = this->blocks[idx]; this->addIdxNoLock(cte_reader_id); + this->total_fetch_block_num++; return CTEOpStatus::OK; } CTEOpStatus CTEPartition::pushBlock(const Block & block) { std::unique_lock aux_lock(*(this->aux_lock)); - if (this->status != CTEPartitionStatus::NORMAL) + CTEOpStatus ret_status = CTEOpStatus::OK; + switch (this->status) { + case CTEPartitionStatus::NEED_SPILL: + ret_status = CTEOpStatus::NEED_SPILL; + case CTEPartitionStatus::IN_SPILLING: + ret_status = CTEOpStatus::WAIT_SPILL; if likely (block.rows() != 0) - { // Block memory usage will be calculated after the finish of spill this->tmp_blocks.push_back(block); - } - return CTEOpStatus::IO_OUT; + return ret_status; + case CTEPartitionStatus::NORMAL: + break; } // mu must be held after aux_lock so that we will not be blocked by spill. // Blocked in cpu pool is very bad. std::lock_guard lock(*this->mu); + this->total_recv_block_num++; + this->total_byte_usage += block.bytes(); + this->memory_usage += block.bytes(); this->blocks.push_back(block); this->pipe_cv->notifyOne(); @@ -72,9 +82,10 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) if unlikely (this->exceedMemoryThresholdNoLock()) { this->setCTEPartitionStatusNoLock(CTEPartitionStatus::NEED_SPILL); - return CTEOpStatus::IO_OUT; + LOG_INFO(this->spill_context->getLog(), "xzxdebug exceed memory, switch to NEED_spill"); + ret_status = CTEOpStatus::NEED_SPILL; } - return CTEOpStatus::OK; + return ret_status; } CTEOpStatus CTEPartition::spillBlocks() @@ -93,7 +104,7 @@ CTEOpStatus CTEPartition::spillBlocks() case CTEPartitionStatus::NORMAL: return CTEOpStatus::OK; case CTEPartitionStatus::IN_SPILLING: - return CTEOpStatus::IO_OUT; + return CTEOpStatus::WAIT_SPILL; case CTEPartitionStatus::NEED_SPILL: this->setCTEPartitionStatusNoLock(CTEPartitionStatus::IN_SPILLING); break; @@ -108,29 +119,33 @@ CTEOpStatus CTEPartition::spillBlocks() this->tmp_blocks.clear(); } - auto cte_reader_num = this->fetch_block_idxs.size(); - std::vector split_idxs{0}; - split_idxs.reserve(cte_reader_num + 1); - for (auto iter : this->fetch_block_idxs) - split_idxs.push_back(iter.second); - std::sort(split_idxs.begin(), split_idxs.end()); - - auto begin_iter = this->blocks.begin(); - auto idx_num = split_idxs.size(); - for (size_t i = 0; i < idx_num; i++) + // Key represents logical index + // Value represents physical index at `this->blocks` + std::map split_idxs; + split_idxs.insert(std::make_pair(this->total_block_in_disk_num, 0)); + for (const auto & [_, logical_idx] : this->fetch_block_idxs) + if (logical_idx > this->total_block_in_disk_num) + split_idxs.insert(std::make_pair(logical_idx, logical_idx - this->total_block_in_disk_num)); + + auto blocks_begin_iter = this->blocks.begin(); + auto split_iter = split_idxs.begin(); + auto total_block_in_memory_num = this->blocks.size(); + while (split_iter != split_idxs.end()) { - if (split_idxs[i] >= this->blocks.size()) - break; + auto next_iter = std::next(split_iter); Blocks spilled_blocks; - if (i == idx_num - 1) - spilled_blocks.assign(begin_iter + split_idxs[i], this->blocks.end()); + if (next_iter == split_idxs.end() || next_iter->second >= total_block_in_memory_num) + spilled_blocks.assign(blocks_begin_iter + split_iter->second, this->blocks.end()); else - spilled_blocks.assign(begin_iter + split_idxs[i], begin_iter + split_idxs[i + 1]); + spilled_blocks.assign(blocks_begin_iter + split_iter->second, blocks_begin_iter + next_iter->second); + this->total_block_in_disk_num += spilled_blocks.size(); + this->total_spill_block_num += spilled_blocks.size(); // TODO remove - auto spiller = this->spill_context->getSpillAt(i); - this->spillers.insert(std::make_pair(split_idxs[i], spiller)); + auto spiller = this->spill_context->getSpiller(this->partition_id, this->spillers.size()); spiller->spillBlocks(std::move(spilled_blocks), this->partition_id); + spiller->finishSpill(); + this->spillers.insert(std::make_pair(split_iter->first, std::move(spiller))); } LOG_INFO( @@ -157,13 +172,8 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) std::unique_lock lock(*(this->mu), std::defer_lock); { std::lock_guard aux_lock(*(this->aux_lock)); - switch (this->status) - { - case CTEPartitionStatus::IN_SPILLING: - return CTEOpStatus::IO_OUT; - default: - break; - } + if (this->status == CTEPartitionStatus::IN_SPILLING) + return CTEOpStatus::WAIT_SPILL; lock.lock(); } @@ -171,6 +181,8 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) RUNTIME_CHECK_MSG(this->isSpillTriggeredNoLock(), "Spill should be triggered"); RUNTIME_CHECK_MSG(this->isBlockAvailableInDiskNoLock(cte_reader_id), "Requested block is not in disk"); + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, fmt::format("xzxdebug try to get block for {}", cte_reader_id)); bool retry = false; do { @@ -179,8 +191,22 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) { auto spiller_iter = this->spillers.find(this->fetch_block_idxs[cte_reader_id]); if (spiller_iter == this->spillers.end()) + { + LOG_INFO( + log, + fmt::format( + "xzxdebug fail to find restore stream {}, {}", + cte_reader_id, + this->fetch_block_idxs[cte_reader_id])); // All blocks in disk have been consumed return CTEOpStatus::OK; + } + LOG_INFO( + log, + fmt::format( + "xzxdebug success to find restore stream {}, {}", + cte_reader_id, + this->fetch_block_idxs[cte_reader_id])); auto streams = spiller_iter->second->restoreBlocks(this->partition_id, 1); RUNTIME_CHECK(streams.size() == 1); iter->second = streams[0]; @@ -188,6 +214,7 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) } block = iter->second->read(); + LOG_INFO(log, fmt::format("xzxdebug read for {}, {}", cte_reader_id, this->fetch_block_idxs[cte_reader_id])); if (!block) { RUNTIME_CHECK(!retry); @@ -195,6 +222,9 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) iter->second->readSuffix(); iter->second = nullptr; retry = true; + LOG_INFO( + log, + fmt::format("xzxdebug retry for {}, {}", cte_reader_id, this->fetch_block_idxs[cte_reader_id])); continue; } diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index cf899988576..17f9c3c4a12 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -36,8 +36,9 @@ enum class CTEOpStatus { OK, BLOCK_NOT_AVAILABLE, // It means that we do not have specified block so far - IO_OUT, + WAIT_SPILL, IO_IN, + NEED_SPILL, END_OF_FILE, CANCELLED }; @@ -51,6 +52,20 @@ struct CTEPartition , aux_lock(std::make_unique()) {} + void debugOutput() + { + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO( + log, + fmt::format( + "xzxdebug CTEPartition total_recv_block_num: {}, total_spill_block_num: {}, total_fetch_block_num: {}, " + "total_byte_usage: {}", + total_recv_block_num, + total_spill_block_num, + total_fetch_block_num, + total_byte_usage)); + } + void init(std::shared_ptr spill_context_, size_t memory_threoshold_) { this->spill_context = spill_context_; @@ -89,6 +104,11 @@ struct CTEPartition return this->isBlockAvailableInMemoryNoLock(cte_reader_id); } + size_t total_recv_block_num = 0; + size_t total_spill_block_num = 0; + size_t total_fetch_block_num = 0; + size_t total_byte_usage = 0; + size_t partition_id; std::unique_ptr mu; @@ -104,7 +124,7 @@ struct CTEPartition Blocks tmp_blocks; std::vector block_in_disk_nums; - std::unordered_map spillers; + std::unordered_map spillers; std::unordered_map cte_reader_restore_streams; UInt64 total_block_in_disk_num = 0; diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index cfbdab7cbdf..e2f619e1ecb 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -17,6 +17,8 @@ #include +#include "Operators/CTEPartition.h" + namespace DB { CTEOpStatus CTEReader::fetchNextBlock(size_t partition_id, Block & block) @@ -30,7 +32,8 @@ CTEOpStatus CTEReader::fetchNextBlock(size_t partition_id, Block & block) if (this->resp.execution_summaries_size() == 0) this->cte->tryToGetResp(this->resp); } - case CTEOpStatus::IO_OUT: + case CTEOpStatus::WAIT_SPILL: + case CTEOpStatus::NEED_SPILL: case CTEOpStatus::IO_IN: case CTEOpStatus::BLOCK_NOT_AVAILABLE: case CTEOpStatus::OK: diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index e71acf161e0..d3215ff3e74 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -31,15 +31,20 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) { if (!block) return OperatorStatus::FINISHED; + auto * log = &Poco::Logger::get("LRUCache"); this->total_rows += block.rows(); auto status = this->cte->pushBlock(this->id, block); switch (status) { - case CTEOpStatus::IO_OUT: + case CTEOpStatus::WAIT_SPILL: + LOG_INFO(log, "xzxdebug CTESinkOp waits for the finish of spill"); // CTE is spilling blocks to disk, we need to wait the finish of spill DB::setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; + case CTEOpStatus::NEED_SPILL: + LOG_INFO(log, "xzxdebug CTESinkOp starts spill"); + return OperatorStatus::IO_OUT; case CTEOpStatus::OK: return OperatorStatus::NEED_INPUT; case CTEOpStatus::CANCELLED: @@ -51,12 +56,15 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) OperatorStatus CTESinkOp::executeIOImpl() { + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, "xzxdebug CTESinkOp enter executeIOImpl"); auto status = this->cte->spillBlocks(this->id); switch (status) { case CTEOpStatus::OK: return OperatorStatus::NEED_INPUT; - case CTEOpStatus::IO_OUT: + case CTEOpStatus::WAIT_SPILL: + LOG_INFO(log, "xzxdebug CTESinkOp waits for spill in executeIOImpl"); // CTE is spilling blocks to disk, we need to wait the finish of spill DB::setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index eed4ddbe39f..b6cadf2e5a7 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -35,6 +35,8 @@ OperatorStatus CTESourceOp::readImpl(Block & block) return OperatorStatus::HAS_OUTPUT; } + auto * log = &Poco::Logger::get("LRUCache"); + auto ret = this->cte_reader->fetchNextBlock(this->id, block); switch (ret) { @@ -42,14 +44,17 @@ OperatorStatus CTESourceOp::readImpl(Block & block) this->cte_reader->getResp(this->resp); if (this->resp.execution_summaries_size() != 0) this->io_profile_info->remote_execution_summary.add(this->resp); + LOG_INFO(log, "xzxdebug encounter eof in CTESourceOp"); case CTEOpStatus::OK: this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; case CTEOpStatus::IO_IN: + LOG_INFO(log, "xzxdebug CTESourceOp is going to read block from disk"); // Expected block is in disk, we need to read it from disk return OperatorStatus::IO_IN; - case CTEOpStatus::IO_OUT: + case CTEOpStatus::WAIT_SPILL: // CTE is spilling blocks to disk, we need to wait the finish of spill + LOG_INFO(log, "xzxdebug waiting for spill in CTESourceOp"); DB::setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::CANCELLED: @@ -64,13 +69,17 @@ OperatorStatus CTESourceOp::readImpl(Block & block) OperatorStatus CTESourceOp::executeIOImpl() { + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, "xzxdebug CTESourceOp enters executeIOImpl"); + RUNTIME_CHECK(!this->block_from_disk); auto status = this->cte_reader->fetchBlockFromDisk(this->id, this->block_from_disk); switch (status) { case CTEOpStatus::OK: return OperatorStatus::HAS_OUTPUT; - case CTEOpStatus::IO_OUT: + case CTEOpStatus::WAIT_SPILL: + LOG_INFO(log, "xzxdebug CTESourceOp wait spill in executeIOImpl"); // CTE is spilling blocks to disk, we need to wait the finish of spill DB::setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; From d20cf301e73ee64ca479bd2d772648cc02613213 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 9 Jul 2025 16:25:12 +0800 Subject: [PATCH 089/118] fix bugs --- dbms/src/Interpreters/CTESpillContext.cpp | 2 +- dbms/src/Operators/CTEPartition.cpp | 46 +- dbms/src/Operators/CTEPartition.h | 19 +- dbms/src/Operators/CTESinkOp.cpp | 6 - dbms/src/Operators/CTESourceOp.cpp | 9 - stack | 32007 ++++++++++++++++++++ 6 files changed, 32046 insertions(+), 43 deletions(-) create mode 100644 stack diff --git a/dbms/src/Interpreters/CTESpillContext.cpp b/dbms/src/Interpreters/CTESpillContext.cpp index 9740be358ce..07a28498854 100644 --- a/dbms/src/Interpreters/CTESpillContext.cpp +++ b/dbms/src/Interpreters/CTESpillContext.cpp @@ -31,6 +31,6 @@ SpillerPtr CTESpillContext::getSpiller(size_t partition_id, size_t spill_id) this->spill_config.for_all_constant_max_streams, this->spill_config.for_all_constant_block_size); - return std::make_unique(config, false, this->partition_num, this->spill_block_schema, this->log); + return std::make_unique(config, false, this->partition_num, this->spill_block_schema, this->log, 1, false); } } // namespace DB diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 7b094d3dbed..0a7b75e6293 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -46,7 +46,14 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) auto idx = this->getIdxInMemoryNoLock(cte_reader_id); block = this->blocks[idx]; this->addIdxNoLock(cte_reader_id); - this->total_fetch_block_num++; + { + auto [iter, _] = this->total_fetch_block_nums.insert(std::make_pair(cte_reader_id, 0)); + iter->second++; + } + { + auto [iter, _] = this->total_fetch_row_nums.insert(std::make_pair(cte_reader_id, 0)); + iter->second += block.rows(); + } return CTEOpStatus::OK; } @@ -73,6 +80,7 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) std::lock_guard lock(*this->mu); this->total_recv_block_num++; + this->total_recv_row_num += block.rows(); this->total_byte_usage += block.bytes(); this->memory_usage += block.bytes(); @@ -82,7 +90,6 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) if unlikely (this->exceedMemoryThresholdNoLock()) { this->setCTEPartitionStatusNoLock(CTEPartitionStatus::NEED_SPILL); - LOG_INFO(this->spill_context->getLog(), "xzxdebug exceed memory, switch to NEED_spill"); ret_status = CTEOpStatus::NEED_SPILL; } return ret_status; @@ -132,6 +139,9 @@ CTEOpStatus CTEPartition::spillBlocks() auto total_block_in_memory_num = this->blocks.size(); while (split_iter != split_idxs.end()) { + if (split_iter->second == this->blocks.size()) + break; + auto next_iter = std::next(split_iter); Blocks spilled_blocks; @@ -139,13 +149,18 @@ CTEOpStatus CTEPartition::spillBlocks() spilled_blocks.assign(blocks_begin_iter + split_iter->second, this->blocks.end()); else spilled_blocks.assign(blocks_begin_iter + split_iter->second, blocks_begin_iter + next_iter->second); + + RUNTIME_CHECK(!spilled_blocks.empty()); + this->total_block_in_disk_num += spilled_blocks.size(); + this->total_spill_block_num += spilled_blocks.size(); // TODO remove auto spiller = this->spill_context->getSpiller(this->partition_id, this->spillers.size()); spiller->spillBlocks(std::move(spilled_blocks), this->partition_id); spiller->finishSpill(); this->spillers.insert(std::make_pair(split_iter->first, std::move(spiller))); + split_iter++; } LOG_INFO( @@ -181,32 +196,18 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) RUNTIME_CHECK_MSG(this->isSpillTriggeredNoLock(), "Spill should be triggered"); RUNTIME_CHECK_MSG(this->isBlockAvailableInDiskNoLock(cte_reader_id), "Requested block is not in disk"); - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, fmt::format("xzxdebug try to get block for {}", cte_reader_id)); bool retry = false; - do + while (true) { auto [iter, _] = this->cte_reader_restore_streams.insert(std::make_pair(cte_reader_id, nullptr)); if (iter->second == nullptr) { auto spiller_iter = this->spillers.find(this->fetch_block_idxs[cte_reader_id]); if (spiller_iter == this->spillers.end()) - { - LOG_INFO( - log, - fmt::format( - "xzxdebug fail to find restore stream {}, {}", - cte_reader_id, - this->fetch_block_idxs[cte_reader_id])); + // All blocks in disk have been consumed return CTEOpStatus::OK; - } - LOG_INFO( - log, - fmt::format( - "xzxdebug success to find restore stream {}, {}", - cte_reader_id, - this->fetch_block_idxs[cte_reader_id])); + auto streams = spiller_iter->second->restoreBlocks(this->partition_id, 1); RUNTIME_CHECK(streams.size() == 1); iter->second = streams[0]; @@ -214,7 +215,6 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) } block = iter->second->read(); - LOG_INFO(log, fmt::format("xzxdebug read for {}, {}", cte_reader_id, this->fetch_block_idxs[cte_reader_id])); if (!block) { RUNTIME_CHECK(!retry); @@ -222,14 +222,12 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) iter->second->readSuffix(); iter->second = nullptr; retry = true; - LOG_INFO( - log, - fmt::format("xzxdebug retry for {}, {}", cte_reader_id, this->fetch_block_idxs[cte_reader_id])); continue; } this->addIdxNoLock(cte_reader_id); - } while (retry); + break; + }; return CTEOpStatus::OK; } diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 17f9c3c4a12..d90eed36c53 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -54,15 +54,26 @@ struct CTEPartition void debugOutput() { + String info_block; + for (const auto & item : this->total_fetch_block_nums) + info_block = fmt::format("{} <{}: {}>", info_block, item.first, item.second); + + String info_row; + for (const auto & item : this->total_fetch_row_nums) + info_row = fmt::format("{} <{}: {}>", info_row, item.first, item.second); + auto * log = &Poco::Logger::get("LRUCache"); LOG_INFO( log, fmt::format( - "xzxdebug CTEPartition total_recv_block_num: {}, total_spill_block_num: {}, total_fetch_block_num: {}, " + "xzxdebug CTEPartition total_recv_block_num: {}, row: {}, total_spill_block_num: {}, " + "total_fetch_block_num: {}, row num: {}, " "total_byte_usage: {}", total_recv_block_num, + total_recv_row_num, total_spill_block_num, - total_fetch_block_num, + info_block, + info_row, total_byte_usage)); } @@ -105,8 +116,10 @@ struct CTEPartition } size_t total_recv_block_num = 0; + size_t total_recv_row_num = 0; size_t total_spill_block_num = 0; - size_t total_fetch_block_num = 0; + std::map total_fetch_block_nums; + std::map total_fetch_row_nums; size_t total_byte_usage = 0; size_t partition_id; diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index d3215ff3e74..71cd10192a7 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -31,19 +31,16 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) { if (!block) return OperatorStatus::FINISHED; - auto * log = &Poco::Logger::get("LRUCache"); this->total_rows += block.rows(); auto status = this->cte->pushBlock(this->id, block); switch (status) { case CTEOpStatus::WAIT_SPILL: - LOG_INFO(log, "xzxdebug CTESinkOp waits for the finish of spill"); // CTE is spilling blocks to disk, we need to wait the finish of spill DB::setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::NEED_SPILL: - LOG_INFO(log, "xzxdebug CTESinkOp starts spill"); return OperatorStatus::IO_OUT; case CTEOpStatus::OK: return OperatorStatus::NEED_INPUT; @@ -56,15 +53,12 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) OperatorStatus CTESinkOp::executeIOImpl() { - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, "xzxdebug CTESinkOp enter executeIOImpl"); auto status = this->cte->spillBlocks(this->id); switch (status) { case CTEOpStatus::OK: return OperatorStatus::NEED_INPUT; case CTEOpStatus::WAIT_SPILL: - LOG_INFO(log, "xzxdebug CTESinkOp waits for spill in executeIOImpl"); // CTE is spilling blocks to disk, we need to wait the finish of spill DB::setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index b6cadf2e5a7..b70d80199d8 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -35,8 +35,6 @@ OperatorStatus CTESourceOp::readImpl(Block & block) return OperatorStatus::HAS_OUTPUT; } - auto * log = &Poco::Logger::get("LRUCache"); - auto ret = this->cte_reader->fetchNextBlock(this->id, block); switch (ret) { @@ -44,17 +42,14 @@ OperatorStatus CTESourceOp::readImpl(Block & block) this->cte_reader->getResp(this->resp); if (this->resp.execution_summaries_size() != 0) this->io_profile_info->remote_execution_summary.add(this->resp); - LOG_INFO(log, "xzxdebug encounter eof in CTESourceOp"); case CTEOpStatus::OK: this->total_rows += block.rows(); return OperatorStatus::HAS_OUTPUT; case CTEOpStatus::IO_IN: - LOG_INFO(log, "xzxdebug CTESourceOp is going to read block from disk"); // Expected block is in disk, we need to read it from disk return OperatorStatus::IO_IN; case CTEOpStatus::WAIT_SPILL: // CTE is spilling blocks to disk, we need to wait the finish of spill - LOG_INFO(log, "xzxdebug waiting for spill in CTESourceOp"); DB::setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::CANCELLED: @@ -69,9 +64,6 @@ OperatorStatus CTESourceOp::readImpl(Block & block) OperatorStatus CTESourceOp::executeIOImpl() { - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, "xzxdebug CTESourceOp enters executeIOImpl"); - RUNTIME_CHECK(!this->block_from_disk); auto status = this->cte_reader->fetchBlockFromDisk(this->id, this->block_from_disk); switch (status) @@ -79,7 +71,6 @@ OperatorStatus CTESourceOp::executeIOImpl() case CTEOpStatus::OK: return OperatorStatus::HAS_OUTPUT; case CTEOpStatus::WAIT_SPILL: - LOG_INFO(log, "xzxdebug CTESourceOp wait spill in executeIOImpl"); // CTE is spilling blocks to disk, we need to wait the finish of spill DB::setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; diff --git a/stack b/stack new file mode 100644 index 00000000000..d00cf80266f --- /dev/null +++ b/stack @@ -0,0 +1,32007 @@ +Thread 1351 (Thread 0x7f903488f640 (LWP 2743138) "default-executo"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x0000558c288f9efe in absl::lts_20211102::synchronization_internal::FutexImpl::WaitUntil (v=0x7f9575a67340, val=0, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/futex.h:104 +#2 0x0000558c288f9bfb in absl::lts_20211102::synchronization_internal::Waiter::Wait (this=0x7f9575a67340, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/waiter.cc:95 +#3 0x0000558c288f9816 in AbslInternalPerThreadSemWait_lts_20211102 (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.cc:93 +#4 0x0000558c288f6bbd in absl::lts_20211102::synchronization_internal::PerThreadSem::Wait (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.h:112 +#5 0x0000558c288efd56 in absl::lts_20211102::Mutex::DecrementSynchSem (mu=0x7f9493fa3060, w=0x7f9575a67300, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:579 +#6 0x0000558c288f4e7a in absl::lts_20211102::CondVar::WaitCommon (this=0x7f9493fa3078, mutex=0x7f9493fa3060, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2559 +#7 0x0000558c288f50aa in absl::lts_20211102::CondVar::Wait (this=0x7f9493fa3078, mu=0x7f9493fa3060) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2594 +#8 0x0000558c2832ff8b in gpr_cv_wait (cv=0x7f9493fa3078, mu=0x7f9493fa3060, abs_deadline=...) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gpr/sync_abseil.cc:85 +#9 0x0000558c27df194f in grpc_core::Executor::ThreadMain (arg=0x7f9493fa3060) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/executor.cc:238 +#10 0x0000558c28339f22 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::operator()(void*) const (this=0x7f903485bc87, v=0x7f938f4e5d50) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:140 +#11 0x0000558c28339e09 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) (v=0x7f938f4e5d50) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:110 +#12 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#13 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1350 (Thread 0x7f903e99e640 (LWP 2743051) "default-executo"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f958386d4a9 in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9583867500 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f9583909ab3 in grpc_core::Executor::ThreadMain(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1349 (Thread 0x7f903f39f640 (LWP 2743046) "default-executo"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f958386d4a9 in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9583867500 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f9583909ab3 in grpc_core::Executor::ThreadMain(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1348 (Thread 0x7f91984aa640 (LWP 2741765) "io_pool"): +#0 0x0000558c276cdd16 in LZ4_count (pIn=0x7f8fd4d49d08 "", pMatch=0x7f8fd4d49d04 "", pInLimit=0x7f8fd4df6882 "") at /DATA/disk3/xzx/tiflash/contrib/lz4/lib/lz4.c:674 +#1 LZ4_compress_generic_validated (cctx=0x7f9198470510, source=0x7f8fd4d0f740 "\004", dest=0x7f8fd408f589 "O\004", inputSize=946503, inputConsumed=0x0, maxOutputSize=0, outputDirective=notLimited, tableType=byU32, dictDirective=noDict, dictIssue=noDictIssue, acceleration=1) at /DATA/disk3/xzx/tiflash/contrib/lz4/lib/lz4.c:1153 +#2 LZ4_compress_generic (cctx=0x7f9198470510, src=0x7f8fd4d0f740 "\004", dst=0x7f8fd408f589 "O\004", srcSize=946503, inputConsumed=0x0, dstCapacity=0, outputDirective=notLimited, tableType=byU32, dictDirective=noDict, dictIssue=noDictIssue, acceleration=1) at /DATA/disk3/xzx/tiflash/contrib/lz4/lib/lz4.c:1339 +#3 LZ4_compress_fast_extState (state=0x7f9198470510, source=0x7f8fd4d0f740 "\004", dest=0x7f8fd408f589 "O\004", inputSize=946503, maxOutputSize=950230, acceleration=1) at /DATA/disk3/xzx/tiflash/contrib/lz4/lib/lz4.c:1357 +#4 0x0000558c276eb34a in LZ4_compress_fast (source=0x7f8fd4d0f740 "\004", dest=0x7f8fd408f589 "O\004", inputSize=946503, maxOutputSize=950230, acceleration=1) at /DATA/disk3/xzx/tiflash/contrib/lz4/lib/lz4.c:1426 +#5 0x0000558c235cc176 in DB::CompressionCodecLZ4::doCompressData (this=0x7f94ce80b258, source=0x7f8fd4d0f740 "\004", source_size=946503, dest=0x7f8fd408f589 "O\004") at /DATA/disk3/xzx/tiflash/dbms/src/IO/Compression/CompressionCodecLZ4.cpp:46 +#6 0x0000558c235ef8d2 in DB::ICompressionCodec::compress (this=0x7f94ce80b258, source=0x7f8fd4d0f740 "\004", source_size=946503, dest=0x7f8fd408f580 "\202*M\003") at /DATA/disk3/xzx/tiflash/dbms/src/IO/Compression/ICompressionCodec.cpp:38 +#7 0x0000558c235ac7d9 in DB::CompressedWriteBuffer::nextImpl (this=0x7f94bb518070) at /DATA/disk3/xzx/tiflash/dbms/src/IO/Compression/CompressedWriteBuffer.cpp:34 +#8 0x0000558c1aa06234 in DB::WriteBuffer::next (this=0x7f94bb518070) at /DATA/disk3/xzx/tiflash/dbms/src/IO/Buffer/WriteBuffer.h:59 +#9 0x0000558c2347e179 in DB::NativeBlockOutputStream::flush (this=0x7f94bb51e000) at /DATA/disk3/xzx/tiflash/dbms/src/DataStreams/NativeBlockOutputStream.cpp:56 +#10 0x0000558c23d34881 in DB::SpillHandler::SpillWriter::finishWrite (this=0x7f94bb518000) at /DATA/disk3/xzx/tiflash/dbms/src/Core/SpillHandler.cpp:57 +#11 0x0000558c23d35c66 in DB::SpillHandler::spillBlocks (this=0x7f91984759b0, blocks=...) at /DATA/disk3/xzx/tiflash/dbms/src/Core/SpillHandler.cpp:162 +#12 0x0000558c23d21c0a in DB::Spiller::spillBlocks (this=0x7f94bb513280, blocks=..., partition_id=0) at /DATA/disk3/xzx/tiflash/dbms/src/Core/Spiller.cpp:226 +#13 0x0000558c25c24f65 in DB::CTEPartition::spillBlocks (this=0x7f93a4cc8000) at /DATA/disk3/xzx/tiflash/dbms/src/Operators/CTEPartition.cpp:146 +#14 0x0000558c25c20471 in DB::CTE::spillBlocks (this=0x7f93a4cd2558, partition_id=0) at /DATA/disk3/xzx/tiflash/dbms/src/Operators/CTE.cpp:94 +#15 0x0000558c25c310ad in DB::CTESinkOp::executeIOImpl (this=0x7f94ac25df00) at /DATA/disk3/xzx/tiflash/dbms/src/Operators/CTESinkOp.cpp:61 +#16 0x0000558c259823be in DB::Operator::executeIO (this=0x7f94ac25df00) at /DATA/disk3/xzx/tiflash/dbms/src/Operators/Operator.cpp:81 +#17 0x0000558c2598120e in DB::PipelineExec::executeIOImpl (this=0x7f93a3cc5230) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp:194 +#18 0x0000558c2597df15 in DB::PipelineExec::executeIO (this=0x7f93a3cc5230) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp:181 +#19 0x0000558c2599a1cf in DB::PipelineTaskBase::runExecuteIO (this=0x7f94ac25e0a0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/Tasks/Impls/PipelineTaskBase.h:92 +#20 0x0000558c25999b3c in DB::PipelineTask::executeIOImpl (this=0x7f94ac25dfe0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/Tasks/Impls/PipelineTask.h:40 +#21 0x0000558c259a99c8 in DB::Task::executeIO (this=0x7f94ac25dfe0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp:140 +#22 0x0000558c259b60ed in DB::IOImpl::exec (task=...) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPoolImpl.h:48 +#23 0x0000558c259b5e29 in DB::TaskThreadPool::handleTask (this=0x7f91cfae6dc0, task=...) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp:105 +#24 0x0000558c259b5a0e in DB::TaskThreadPool::doLoop (this=0x7f91cfae6dc0, thread_no=0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp:84 +#25 0x0000558c259b5261 in DB::TaskThreadPool::loop (this=0x7f91cfae6dc0, thread_no=0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp:63 +#26 0x0000558c259b71a4 in std::__1::__invoke[abi:ue170006]::*)(unsigned long), DB::TaskThreadPool*, unsigned long, void>(void (DB::TaskThreadPool::*&&)(unsigned long), DB::TaskThreadPool*&&, unsigned long&&) (__f=@0x7f91cf72b358: (void (DB::TaskThreadPool::*)(DB::TaskThreadPool * const, unsigned long)) 0x558c259b5220 ::loop(unsigned long)>, __a0=@0x7f91cf72b368: 0x7f91cfae6dc0, __args=@0x7f91cf72b370: 0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#27 0x0000558c259b711f in std::__1::__thread_execute[abi:ue170006] >, void (DB::TaskThreadPool::*)(unsigned long), DB::TaskThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, void (DB::TaskThreadPool::*)(unsigned long), DB::TaskThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#28 0x0000558c259b6ed2 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::TaskThreadPool::*)(unsigned long), DB::TaskThreadPool*, unsigned long> >(void*) (__vp=0x7f91cf72b350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1347 (Thread 0x7f91c5ef3640 (LWP 2741688) "TiFlashMain"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09f968, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f956f09f968, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c268e60b9 in std::__1::condition_variable::wait_until >, DB::MockLocalAdmissionController::refillTokenBucket()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::MockLocalAdmissionController::refillTokenBucket()::$_0) (this=0x7f956f09f968, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c268e5ddf in std::__1::condition_variable::wait_for[abi:ue170006], DB::MockLocalAdmissionController::refillTokenBucket()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::MockLocalAdmissionController::refillTokenBucket()::$_0) (this=0x7f956f09f968, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 +#6 0x0000558c268e5aec in DB::MockLocalAdmissionController::refillTokenBucket (this=0x7f956f09f940) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/ResourceControl/MockLocalAdmissionController.cpp:25 +#7 0x0000558c1ab8c608 in DB::MockLocalAdmissionController::MockLocalAdmissionController()::{lambda()#1}::operator()() const (this=0x7f91cf5e7508) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/ResourceControl/MockLocalAdmissionController.h:45 +#8 0x0000558c1ab8c5c5 in std::__1::__invoke[abi:ue170006](DB::MockLocalAdmissionController::MockLocalAdmissionController()::{lambda()#1}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#9 0x0000558c1ab8c59d in std::__1::__thread_execute[abi:ue170006] >, DB::MockLocalAdmissionController::MockLocalAdmissionController()::{lambda()#1}>(std::__1::tuple >, DB::MockLocalAdmissionController::MockLocalAdmissionController()::{lambda()#1}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#10 0x0000558c1ab8c3d2 in std::__1::__thread_proxy[abi:ue170006] >, DB::MockLocalAdmissionController::MockLocalAdmissionController()::{lambda()#1}> >(void*) (__vp=0x7f91cf5e7500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#11 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#12 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1346 (Thread 0x7f91c85f4640 (LWP 2741348) "ReadIndexWkr-0"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09fb40, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f956f09fb40, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c26b22249 in DB::AsyncWaker::Notifier::blockedWaitUtil (this=0x7f956f09fac0, time_point=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/Read/AsyncNotifier.cpp:30 +#5 0x0000558c237572f0 in DB::AsyncNotifier::blockedWaitFor (this=0x7f956f09fac0, duration=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/Utils.h:97 +#6 0x0000558c26b71299 in DB::ReadIndexWorkerManager::ReadIndexRunner::blockedWaitFor (this=0x7f9493f3e400, timeout=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/Read/ReadIndexWorkerManager.cpp:140 +#7 0x0000558c26b72450 in DB::ReadIndexWorkerManager::ReadIndexRunner::asyncRun()::$_0::operator()() const (this=0x7f94936ee698) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/Read/ReadIndexWorkerManager.cpp:166 +#8 0x0000558c26b71e05 in std::__1::__invoke[abi:ue170006](DB::ReadIndexWorkerManager::ReadIndexRunner::asyncRun()::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#9 0x0000558c26b71ddd in std::__1::__thread_execute[abi:ue170006] >, DB::ReadIndexWorkerManager::ReadIndexRunner::asyncRun()::$_0>(std::__1::tuple >, DB::ReadIndexWorkerManager::ReadIndexRunner::asyncRun()::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#10 0x0000558c26b71c12 in std::__1::__thread_proxy[abi:ue170006] >, DB::ReadIndexWorkerManager::ReadIndexRunner::asyncRun()::$_0> >(void*) (__vp=0x7f94936ee690) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#11 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#12 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1345 (Thread 0x7f91c87f5640 (LWP 2741347) "RaftStoreProxy"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957dfc02bb in crossbeam_channel::context::Context::wait_until (self=0x7f91c87c14d8, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957e132377 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1} (cx=0x7f91c87c14d8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957dfc0c52 in crossbeam_channel::context::{impl#0}::with::{closure#0}, ()> (cx=0x7f91c87c14d8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957dfc0ba3 in crossbeam_channel::context::{impl#0}::with::{closure#1}, ()> (cell=0x7f91c87f1798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957e43dcab in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, ()>, ()> (self=0x7f9583d91288, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957dfc050e in crossbeam_channel::context::Context::with, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957e13214b in crossbeam_channel::flavors::list::Channel::recv (self=0x7f956a8c7e00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957dfbfaee in crossbeam_channel::channel::Receiver::recv (self=0x7f91c87c1980) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957e3fff9b in tikv_util::mpsc::Receiver::recv (self=0x7f91c87c1980) at components/tikv_util/src/mpsc/mod.rs:153 +#13 0x00007f957c3ab81a in proxy_server::run::run_impl::{closure#0} () at proxy_components/proxy_server/src/run.rs:269 +#14 0x00007f957a80ae27 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#15 0x00007f957cb7a7f6 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#16 0x00007f957c9e6769 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#17 0x00007f957bdf3add in std::panicking::try::do_call, ()>>, ()> (data=0x7f91c87c1a50) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#18 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#19 0x00007f957bdd9b8e in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#20 0x00007f957cb78daf in std::panic::catch_unwind, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#21 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#22 0x00007f957d67f3bf in core::ops::function::FnOnce::call_once, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#23 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#24 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#25 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#26 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#27 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1344 (Thread 0x7f91c89f6640 (LWP 2741346) "status-server"): +#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 +#1 0x00007f9583185778 in mio::sys::unix::selector::epoll::Selector::select (self=0x7f91ea7113b8, events=0x7f91ea711398, timeout=...) at src/sys/unix/selector/epoll.rs:97 +#2 0x00007f9583184fd1 in mio::poll::Poll::poll (self=0x7f91ea7113b0, events=0x7f91ea711398, timeout=...) at src/poll.rs:426 +#3 0x00007f9583167b7c in tokio::runtime::io::driver::Driver::turn (self=0x7f91ea711398, handle=0x7f91ea71e118, max_wait=...) at src/runtime/io/driver.rs:149 +#4 0x00007f95831677ee in tokio::runtime::io::driver::Driver::park (self=0x7f91ea711398, rt_handle=0x7f91ea71e118) at src/runtime/io/driver.rs:122 +#5 0x00007f958316a887 in tokio::runtime::signal::Driver::park (self=0x7f91ea711398, handle=0x7f91ea71e118) at src/runtime/signal/mod.rs:92 +#6 0x00007f9583152787 in tokio::runtime::process::Driver::park (self=0x7f91ea711398, handle=0x7f91ea71e118) at src/runtime/process.rs:32 +#7 0x00007f95830f1aca in tokio::runtime::driver::IoStack::park (self=0x7f91ea711398, handle=0x7f91ea71e118) at src/runtime/driver.rs:175 +#8 0x00007f958316a02e in tokio::runtime::time::Driver::park_internal (self=0x7f91ea711398, rt_handle=0x7f91ea71e118, limit=...) at src/runtime/time/mod.rs:235 +#9 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f91ea711398, handle=0x7f91ea71e118) at src/runtime/time/mod.rs:165 +#10 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f91ea711390, handle=0x7f91ea71e118) at src/runtime/driver.rs:332 +#11 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f91ea711390, handle=0x7f91ea71e118) at src/runtime/driver.rs:71 +#12 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f91ea679620, driver=0x7f91ea711390, handle=0x7f91ea71e118) at src/runtime/scheduler/multi_thread/park.rs:184 +#13 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea679620, handle=0x7f91ea71e118) at src/runtime/scheduler/multi_thread/park.rs:117 +#14 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c89c19a0, handle=0x7f91ea71e118) at src/runtime/scheduler/multi_thread/park.rs:67 +#15 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c89c1e98, core=0x7f956917b940, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#16 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c89c1e98, core=0x7f956917b940) at src/runtime/scheduler/multi_thread/worker.rs:702 +#17 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c89c1e98, core=0x7f956917b940) at src/runtime/scheduler/multi_thread/worker.rs:553 +#18 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#19 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c89c3610, t=0x7f91c89c1e90, f=...) at src/runtime/context/scoped.rs:40 +#20 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c89c35d8) at src/runtime/context.rs:180 +#21 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#22 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#23 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c89c1e90, f=...) at src/runtime/context.rs:180 +#24 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#25 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c89c2098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#26 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#27 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#28 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c89c2220) at src/runtime/blocking/task.rs:42 +#29 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea7114a8) at src/runtime/task/core.rs:328 +#30 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea7114a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#31 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea7114a0, cx=...) at src/runtime/task/core.rs:317 +#32 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#33 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#34 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c89c2398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#35 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#36 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#37 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#38 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea7114a0, cx=...) at src/runtime/task/harness.rs:473 +#39 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c89c25b0) at src/runtime/task/harness.rs:208 +#40 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#41 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#42 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#43 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#44 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#45 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132ded10, worker_thread_id=1) at src/runtime/blocking/pool.rs:513 +#46 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#47 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#48 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#49 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#50 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c89c2a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#51 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#52 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#53 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#54 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#55 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#56 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#57 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#58 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#59 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#60 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1343 (Thread 0x7f91c8bf7640 (LWP 2741345) "status-server"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c8bf3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c8bf3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c8bf3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264679642576, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea6795d0, mutex=0x7f91ea6795d8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea6795d0, mutex_guard=0x7f91c8bc2628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea6795d0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea6795c0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea6795c0, handle=0x7f91ea71e118) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c8bc29a0, handle=0x7f91ea71e118) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c8bc2e98, core=0x7f956917b8f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c8bc2e98, core=0x7f956917b8f0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c8bc2e98, core=0x7f956917b8f0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c8bc4610, t=0x7f91c8bc2e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c8bc45d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c8bc2e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c8bc3098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c8bc3220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea711428) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea711428, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea711420, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c8bc3398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea711420, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c8bc35b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132ded10, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c8bc3a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1342 (Thread 0x7f91c9078640 (LWP 2741344) "snap-sender"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9074750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9074750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9074730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264675270480, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea24df50, mutex=0x7f91ea24df58, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea24df50, mutex_guard=0x7f91c9043628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea24df50, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea24df40) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea24df40, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c90439a0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9043e98, core=0x7f956917b350, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9043e98, core=0x7f956917b350) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9043e98, core=0x7f956917b350) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9045610, t=0x7f91c9043e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c90455d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9043e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9044098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9044220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea314128) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea314128, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea314120, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9044398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea314120, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c90445b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f9247e09150, worker_thread_id=3) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9044a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1341 (Thread 0x7f91c9279640 (LWP 2741343) "snap-sender"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9275750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9275750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9275730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264675270672, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea24e010, mutex=0x7f91ea24e018, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea24e010, mutex_guard=0x7f91c9244628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea24e010, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea24e000) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea24e000, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c92449a0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9244e98, core=0x7f956917b3f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9244e98, core=0x7f956917b3f0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9244e98, core=0x7f956917b3f0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9246610, t=0x7f91c9244e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c92465d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9244e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9245098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9245220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea314228) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea314228, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea314220, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9245398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea314220, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c92455b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f9247e09150, worker_thread_id=2) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9245a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1340 (Thread 0x7f91c947a640 (LWP 2741342) "snap-sender"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9476750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9476750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9476730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264675270576, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea24dfb0, mutex=0x7f91ea24dfb8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea24dfb0, mutex_guard=0x7f91c9445628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea24dfb0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea24dfa0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea24dfa0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c94459a0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9445e98, core=0x7f956917b3a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9445e98, core=0x7f956917b3a0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9445e98, core=0x7f956917b3a0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9447610, t=0x7f91c9445e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c94475d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9445e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9446098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9446220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea3141a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea3141a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea3141a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9446398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea3141a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c94465b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f9247e09150, worker_thread_id=1) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9446a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1339 (Thread 0x7f91c967b640 (LWP 2741341) "snap-sender"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9677750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9677750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9677730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264675270232, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea24de58, mutex=0x7f91ea24de60, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea24de58, mutex_guard=0x7f91c9646368) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea24de58, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830ef2f6 in tokio::runtime::park::Inner::park (self=0x7f91ea24de50) at src/runtime/park.rs:116 +#10 0x00007f95830ef113 in tokio::runtime::park::ParkThread::park (self=0x7f91ea314020) at src/runtime/park.rs:62 +#11 0x00007f95830f1adf in tokio::runtime::driver::IoStack::park (self=0x7f91ea314018, handle=0x7f91ea093d18) at src/runtime/driver.rs:176 +#12 0x00007f95830f2090 in tokio::runtime::driver::TimeDriver::park (self=0x7f91ea314010, handle=0x7f91ea093d18) at src/runtime/driver.rs:333 +#13 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f91ea314010, handle=0x7f91ea093d18) at src/runtime/driver.rs:71 +#14 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f91ea24dee0, driver=0x7f91ea314010, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:184 +#15 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea24dee0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:117 +#16 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c96469a0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:67 +#17 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9646e98, core=0x7f956917b300, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#18 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9646e98, core=0x7f956917b300) at src/runtime/scheduler/multi_thread/worker.rs:702 +#19 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9646e98, core=0x7f956917b300) at src/runtime/scheduler/multi_thread/worker.rs:553 +#20 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#21 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9648610, t=0x7f91c9646e90, f=...) at src/runtime/context/scoped.rs:40 +#22 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c96485d8) at src/runtime/context.rs:180 +#23 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#24 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#25 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9646e90, f=...) at src/runtime/context.rs:180 +#26 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#27 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9647098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#28 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#29 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#30 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9647220) at src/runtime/blocking/task.rs:42 +#31 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea3140a8) at src/runtime/task/core.rs:328 +#32 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea3140a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#33 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea3140a0, cx=...) at src/runtime/task/core.rs:317 +#34 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#35 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#36 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9647398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#37 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#38 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#39 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#40 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea3140a0, cx=...) at src/runtime/task/harness.rs:473 +#41 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c96475b0) at src/runtime/task/harness.rs:208 +#42 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#43 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#44 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#45 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#46 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#47 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f9247e09150, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 +#48 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#49 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#50 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#51 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#52 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9647a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#53 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#54 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#55 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#56 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#57 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#58 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#59 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#60 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#61 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#62 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1338 (Thread 0x7f91c987c640 (LWP 2741340) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9878750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9878750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9878730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674687776, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bfb20, mutex=0x7f91ea1bfb28, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bfb20, mutex_guard=0x7f91c9847628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bfb20, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bfb10) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bfb10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c98479a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9847e98, core=0x7f956917afe0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9847e98, core=0x7f956917afe0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9847e98, core=0x7f956917afe0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9849610, t=0x7f91c9847e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c98495d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9847e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9848098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9848220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21ff28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21ff28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21ff20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9848398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21ff20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c98485b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=143) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9848a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1337 (Thread 0x7f91c9a7d640 (LWP 2741339) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9a79750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9a79750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9a79730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674687680, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bfac0, mutex=0x7f91ea1bfac8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bfac0, mutex_guard=0x7f91c9a48628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bfac0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bfab0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bfab0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c9a489a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9a48e98, core=0x7f956917af90, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9a48e98, core=0x7f956917af90) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9a48e98, core=0x7f956917af90) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9a4a610, t=0x7f91c9a48e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c9a4a5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9a48e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9a49098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9a49220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fea8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fea8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fea0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9a49398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fea0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c9a495b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=142) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9a49a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1336 (Thread 0x7f91c9c7e640 (LWP 2741338) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9c7a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9c7a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9c7a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674687584, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bfa60, mutex=0x7f91ea1bfa68, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bfa60, mutex_guard=0x7f91c9c49628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bfa60, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bfa50) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bfa50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c9c499a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9c49e98, core=0x7f956917af40, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9c49e98, core=0x7f956917af40) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9c49e98, core=0x7f956917af40) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9c4b610, t=0x7f91c9c49e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c9c4b5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9c49e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9c4a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9c4a220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fe28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fe28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fe20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9c4a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fe20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c9c4a5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=141) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9c4aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1335 (Thread 0x7f91c9e7f640 (LWP 2741337) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9e7b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9e7b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9e7b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674687488, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bfa00, mutex=0x7f91ea1bfa08, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bfa00, mutex_guard=0x7f91c9e4a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bfa00, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf9f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf9f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c9e4a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9e4ae98, core=0x7f956917aef0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9e4ae98, core=0x7f956917aef0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9e4ae98, core=0x7f956917aef0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9e4c610, t=0x7f91c9e4ae90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c9e4c5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9e4ae90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9e4b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9e4b220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fda8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fda8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fda0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9e4b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fda0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c9e4b5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=140) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9e4ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1334 (Thread 0x7f91ca080640 (LWP 2741336) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ca07c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ca07c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ca07c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674687392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf9a0, mutex=0x7f91ea1bf9a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf9a0, mutex_guard=0x7f91ca04b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf9a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf990) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf990, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ca04b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ca04be98, core=0x7f956917aea0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ca04be98, core=0x7f956917aea0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ca04be98, core=0x7f956917aea0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ca04d610, t=0x7f91ca04be90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ca04d5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ca04be90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ca04c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ca04c220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fd28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fd28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fd20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ca04c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fd20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ca04c5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=139) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ca04ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1333 (Thread 0x7f91ca281640 (LWP 2741335) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ca27d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ca27d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ca27d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674687296, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf940, mutex=0x7f91ea1bf948, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf940, mutex_guard=0x7f91ca24c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf940, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf930) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf930, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ca24c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ca24ce98, core=0x7f956917ae50, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ca24ce98, core=0x7f956917ae50) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ca24ce98, core=0x7f956917ae50) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ca24e610, t=0x7f91ca24ce90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ca24e5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ca24ce90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ca24d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ca24d220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fca8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fca8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fca0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ca24d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fca0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ca24d5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=138) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ca24da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1332 (Thread 0x7f91ca482640 (LWP 2741334) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ca47e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ca47e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ca47e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674687200, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf8e0, mutex=0x7f91ea1bf8e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf8e0, mutex_guard=0x7f91ca44d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf8e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf8d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf8d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ca44d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ca44de98, core=0x7f956917ae00, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ca44de98, core=0x7f956917ae00) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ca44de98, core=0x7f956917ae00) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ca44f610, t=0x7f91ca44de90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ca44f5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ca44de90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ca44e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ca44e220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fc28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fc28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fc20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ca44e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fc20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ca44e5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=137) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ca44ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1331 (Thread 0x7f91ca683640 (LWP 2741333) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ca67f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ca67f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ca67f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674687104, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf880, mutex=0x7f91ea1bf888, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf880, mutex_guard=0x7f91ca64e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf880, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf870) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf870, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ca64e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ca64ee98, core=0x7f956917adb0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ca64ee98, core=0x7f956917adb0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ca64ee98, core=0x7f956917adb0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ca650610, t=0x7f91ca64ee90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ca6505d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ca64ee90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ca64f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ca64f220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fba8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fba8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fba0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ca64f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fba0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ca64f5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=136) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ca64fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1330 (Thread 0x7f91ca884640 (LWP 2741332) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ca880750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ca880750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ca880730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674687008, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf820, mutex=0x7f91ea1bf828, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf820, mutex_guard=0x7f91ca84f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf820, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf810) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf810, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ca84f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ca84fe98, core=0x7f956917ad60, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ca84fe98, core=0x7f956917ad60) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ca84fe98, core=0x7f956917ad60) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ca851610, t=0x7f91ca84fe90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ca8515d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ca84fe90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ca850098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ca850220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fb28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fb28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fb20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ca850398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fb20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ca8505b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=135) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ca850a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1329 (Thread 0x7f91caa85640 (LWP 2741331) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91caa81750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91caa81750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91caa81730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674686912, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf7c0, mutex=0x7f91ea1bf7c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf7c0, mutex_guard=0x7f91caa50628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf7c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf7b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf7b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91caa509a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91caa50e98, core=0x7f956917ad10, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91caa50e98, core=0x7f956917ad10) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91caa50e98, core=0x7f956917ad10) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91caa52610, t=0x7f91caa50e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91caa525d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91caa50e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91caa51098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91caa51220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21faa8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21faa8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21faa0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91caa51398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21faa0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91caa515b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=134) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91caa51a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1328 (Thread 0x7f91cac86640 (LWP 2741330) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cac82750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cac82750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cac82730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674686816, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf760, mutex=0x7f91ea1bf768, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf760, mutex_guard=0x7f91cac51628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf760, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf750) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf750, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cac519a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cac51e98, core=0x7f956917acc0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cac51e98, core=0x7f956917acc0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cac51e98, core=0x7f956917acc0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cac53610, t=0x7f91cac51e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cac535d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cac51e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cac52098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cac52220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fa28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fa28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fa20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cac52398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fa20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cac525b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=133) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cac52a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1327 (Thread 0x7f91cae87640 (LWP 2741329) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cae83750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cae83750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cae83730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674686720, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf700, mutex=0x7f91ea1bf708, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf700, mutex_guard=0x7f91cae52628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf700, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf6f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf6f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cae529a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cae52e98, core=0x7f956917ac70, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cae52e98, core=0x7f956917ac70) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cae52e98, core=0x7f956917ac70) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cae54610, t=0x7f91cae52e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cae545d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cae52e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cae53098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cae53220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f9a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f9a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f9a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cae53398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f9a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cae535b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=132) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cae53a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1326 (Thread 0x7f91cb088640 (LWP 2741328) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cb084750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cb084750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cb084730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674686624, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf6a0, mutex=0x7f91ea1bf6a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf6a0, mutex_guard=0x7f91cb053628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf6a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf690) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf690, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cb0539a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cb053e98, core=0x7f956917ac20, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cb053e98, core=0x7f956917ac20) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cb053e98, core=0x7f956917ac20) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cb055610, t=0x7f91cb053e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cb0555d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cb053e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cb054098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cb054220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f928) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f928, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f920, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cb054398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f920, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cb0545b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=131) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cb054a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1325 (Thread 0x7f91cb289640 (LWP 2741327) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cb285750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cb285750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cb285730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674686528, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf640, mutex=0x7f91ea1bf648, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf640, mutex_guard=0x7f91cb254628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf640, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf630) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf630, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cb2549a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cb254e98, core=0x7f956917abd0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cb254e98, core=0x7f956917abd0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cb254e98, core=0x7f956917abd0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cb256610, t=0x7f91cb254e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cb2565d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cb254e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cb255098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cb255220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f8a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f8a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f8a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cb255398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f8a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cb2555b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=130) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cb255a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1324 (Thread 0x7f91cb48a640 (LWP 2741326) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cb486750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cb486750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cb486730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674686432, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf5e0, mutex=0x7f91ea1bf5e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf5e0, mutex_guard=0x7f91cb455628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf5e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf5d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf5d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cb4559a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cb455e98, core=0x7f956917ab80, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cb455e98, core=0x7f956917ab80) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cb455e98, core=0x7f956917ab80) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cb457610, t=0x7f91cb455e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cb4575d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cb455e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cb456098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cb456220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f828) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f828, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f820, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cb456398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f820, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cb4565b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=129) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cb456a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1323 (Thread 0x7f91cb68b640 (LWP 2741325) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cb687750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cb687750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cb687730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674686336, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf580, mutex=0x7f91ea1bf588, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf580, mutex_guard=0x7f91cb656628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf580, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf570) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf570, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cb6569a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cb656e98, core=0x7f956917ab30, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cb656e98, core=0x7f956917ab30) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cb656e98, core=0x7f956917ab30) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cb658610, t=0x7f91cb656e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cb6585d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cb656e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cb657098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cb657220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f7a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f7a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f7a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cb657398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f7a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cb6575b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=128) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cb657a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1322 (Thread 0x7f91cb88c640 (LWP 2741324) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cb888750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cb888750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cb888730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674686240, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf520, mutex=0x7f91ea1bf528, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf520, mutex_guard=0x7f91cb857628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf520, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf510) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf510, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cb8579a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cb857e98, core=0x7f956917aae0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cb857e98, core=0x7f956917aae0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cb857e98, core=0x7f956917aae0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cb859610, t=0x7f91cb857e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cb8595d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cb857e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cb858098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cb858220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f728) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f728, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f720, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cb858398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f720, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cb8585b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=127) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cb858a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1321 (Thread 0x7f91cba8d640 (LWP 2741323) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cba89750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cba89750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cba89730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674686144, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf4c0, mutex=0x7f91ea1bf4c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf4c0, mutex_guard=0x7f91cba58628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf4c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf4b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf4b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cba589a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cba58e98, core=0x7f956917aa90, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cba58e98, core=0x7f956917aa90) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cba58e98, core=0x7f956917aa90) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cba5a610, t=0x7f91cba58e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cba5a5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cba58e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cba59098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cba59220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f6a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f6a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f6a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cba59398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f6a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cba595b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=126) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cba59a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1320 (Thread 0x7f91cbc8e640 (LWP 2741322) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cbc8a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cbc8a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cbc8a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674686048, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf460, mutex=0x7f91ea1bf468, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf460, mutex_guard=0x7f91cbc59628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf460, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf450) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf450, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cbc599a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cbc59e98, core=0x7f956917aa40, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cbc59e98, core=0x7f956917aa40) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cbc59e98, core=0x7f956917aa40) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cbc5b610, t=0x7f91cbc59e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cbc5b5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cbc59e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cbc5a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cbc5a220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f628) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f628, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f620, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cbc5a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f620, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cbc5a5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=125) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cbc5aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1319 (Thread 0x7f91cbe8f640 (LWP 2741321) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cbe8b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cbe8b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cbe8b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674685856, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf3a0, mutex=0x7f91ea1bf3a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf3a0, mutex_guard=0x7f91cbe5a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf3a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf390) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf390, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cbe5a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cbe5ae98, core=0x7f956917a9a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cbe5ae98, core=0x7f956917a9a0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cbe5ae98, core=0x7f956917a9a0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cbe5c610, t=0x7f91cbe5ae90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cbe5c5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cbe5ae90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cbe5b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cbe5b220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f528) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f528, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f520, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cbe5b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f520, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cbe5b5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=124) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cbe5ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1318 (Thread 0x7f91cc090640 (LWP 2741320) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cc08c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cc08c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cc08c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674685952, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf400, mutex=0x7f91ea1bf408, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf400, mutex_guard=0x7f91cc05b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf400, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf3f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf3f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cc05b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cc05be98, core=0x7f956917a9f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cc05be98, core=0x7f956917a9f0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cc05be98, core=0x7f956917a9f0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cc05d610, t=0x7f91cc05be90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cc05d5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cc05be90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cc05c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cc05c220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f5a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f5a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f5a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cc05c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f5a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cc05c5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=123) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cc05ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1317 (Thread 0x7f91cc291640 (LWP 2741319) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cc28d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cc28d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cc28d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674685760, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf340, mutex=0x7f91ea1bf348, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf340, mutex_guard=0x7f91cc25c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf340, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf330) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf330, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cc25c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cc25ce98, core=0x7f956917a950, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cc25ce98, core=0x7f956917a950) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cc25ce98, core=0x7f956917a950) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cc25e610, t=0x7f91cc25ce90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cc25e5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cc25ce90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cc25d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cc25d220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2184a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2184a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2184a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cc25d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2184a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cc25d5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=122) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cc25da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1316 (Thread 0x7f91cc492640 (LWP 2741318) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cc48e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cc48e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cc48e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674685664, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf2e0, mutex=0x7f91ea1bf2e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf2e0, mutex_guard=0x7f91cc45d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf2e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf2d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf2d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cc45d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cc45de98, core=0x7f956917a900, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cc45de98, core=0x7f956917a900) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cc45de98, core=0x7f956917a900) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cc45f610, t=0x7f91cc45de90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cc45f5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cc45de90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cc45e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cc45e220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea218428) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea218428, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea218420, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cc45e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea218420, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cc45e5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=121) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cc45ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1315 (Thread 0x7f91cc693640 (LWP 2741317) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cc68f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cc68f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cc68f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674685568, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf280, mutex=0x7f91ea1bf288, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf280, mutex_guard=0x7f91cc65e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf280, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf270) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf270, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cc65e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cc65ee98, core=0x7f956917a8b0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cc65ee98, core=0x7f956917a8b0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cc65ee98, core=0x7f956917a8b0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cc660610, t=0x7f91cc65ee90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cc6605d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cc65ee90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cc65f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cc65f220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2183a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2183a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2183a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cc65f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2183a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cc65f5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=120) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cc65fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1314 (Thread 0x7f91cc894640 (LWP 2741316) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cc890750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cc890750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cc890730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674685472, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf220, mutex=0x7f91ea1bf228, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf220, mutex_guard=0x7f91cc85f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf220, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf210) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf210, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cc85f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cc85fe98, core=0x7f956917a860, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cc85fe98, core=0x7f956917a860) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cc85fe98, core=0x7f956917a860) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cc861610, t=0x7f91cc85fe90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cc8615d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cc85fe90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cc860098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cc860220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea218328) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea218328, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea218320, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cc860398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea218320, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cc8605b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=119) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cc860a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1313 (Thread 0x7f91cca95640 (LWP 2741315) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cca91750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cca91750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cca91730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674685280, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf160, mutex=0x7f91ea1bf168, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf160, mutex_guard=0x7f91cca60628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf160, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf150) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf150, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cca609a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cca60e98, core=0x7f956917a7c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cca60e98, core=0x7f956917a7c0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cca60e98, core=0x7f956917a7c0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cca62610, t=0x7f91cca60e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cca625d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cca60e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cca61098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cca61220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea218228) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea218228, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea218220, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cca61398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea218220, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cca615b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=118) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cca61a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1312 (Thread 0x7f91ccc96640 (LWP 2741314) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ccc92750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ccc92750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ccc92730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674685376, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf1c0, mutex=0x7f91ea1bf1c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf1c0, mutex_guard=0x7f91ccc61628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf1c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf1b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf1b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ccc619a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ccc61e98, core=0x7f956917a810, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ccc61e98, core=0x7f956917a810) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ccc61e98, core=0x7f956917a810) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ccc63610, t=0x7f91ccc61e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ccc635d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ccc61e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ccc62098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ccc62220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2182a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2182a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2182a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ccc62398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2182a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ccc625b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=117) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ccc62a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1311 (Thread 0x7f91cce97640 (LWP 2741313) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cce93750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cce93750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cce93730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674685184, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf100, mutex=0x7f91ea1bf108, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf100, mutex_guard=0x7f91cce62628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf100, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf0f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf0f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cce629a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cce62e98, core=0x7f956917a770, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cce62e98, core=0x7f956917a770) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cce62e98, core=0x7f956917a770) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cce64610, t=0x7f91cce62e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cce645d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cce62e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cce63098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cce63220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2181a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2181a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2181a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cce63398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2181a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cce635b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=116) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cce63a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1310 (Thread 0x7f91cd098640 (LWP 2741312) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cd094750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cd094750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cd094730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674685088, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf0a0, mutex=0x7f91ea1bf0a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf0a0, mutex_guard=0x7f91cd063628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf0a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf090) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf090, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cd0639a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cd063e98, core=0x7f956917a720, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cd063e98, core=0x7f956917a720) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cd063e98, core=0x7f956917a720) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cd065610, t=0x7f91cd063e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cd0655d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cd063e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cd064098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cd064220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea218128) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea218128, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea218120, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cd064398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea218120, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cd0645b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=115) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cd064a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1309 (Thread 0x7f91cd299640 (LWP 2741311) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cd295750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cd295750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cd295730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684992, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf040, mutex=0x7f91ea1bf048, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf040, mutex_guard=0x7f91cd264628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf040, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf030) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf030, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cd2649a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cd264e98, core=0x7f956917a6d0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cd264e98, core=0x7f956917a6d0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cd264e98, core=0x7f956917a6d0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cd266610, t=0x7f91cd264e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cd2665d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cd264e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cd265098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cd265220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2180a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2180a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2180a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cd265398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2180a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cd2655b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=114) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cd265a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1308 (Thread 0x7f91cd49a640 (LWP 2741310) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cd496750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cd496750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cd496730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684896, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1befe0, mutex=0x7f91ea1befe8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1befe0, mutex_guard=0x7f91cd465628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1befe0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1befd0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1befd0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cd4659a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cd465e98, core=0x7f956917a680, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cd465e98, core=0x7f956917a680) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cd465e98, core=0x7f956917a680) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cd467610, t=0x7f91cd465e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cd4675d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cd465e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cd466098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cd466220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea218028) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea218028, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea218020, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cd466398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea218020, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cd4665b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=113) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cd466a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1307 (Thread 0x7f91cd69b640 (LWP 2741309) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cd697750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cd697750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cd697730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684800, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bef80, mutex=0x7f91ea1bef88, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bef80, mutex_guard=0x7f91cd666628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bef80, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bef70) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bef70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cd6669a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cd666e98, core=0x7f956917a630, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cd666e98, core=0x7f956917a630) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cd666e98, core=0x7f956917a630) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cd668610, t=0x7f91cd666e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cd6685d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cd666e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cd667098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cd667220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211fa8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211fa8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211fa0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cd667398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211fa0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cd6675b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=112) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cd667a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1306 (Thread 0x7f91cd89c640 (LWP 2741308) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cd898750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cd898750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cd898730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684704, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bef20, mutex=0x7f91ea1bef28, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bef20, mutex_guard=0x7f91cd867628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bef20, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bef10) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bef10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cd8679a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cd867e98, core=0x7f956917a5e0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cd867e98, core=0x7f956917a5e0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cd867e98, core=0x7f956917a5e0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cd869610, t=0x7f91cd867e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cd8695d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cd867e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cd868098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cd868220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211f28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211f28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211f20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cd868398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211f20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cd8685b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=111) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cd868a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1305 (Thread 0x7f91cda9d640 (LWP 2741307) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cda99750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cda99750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cda99730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684608, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1beec0, mutex=0x7f91ea1beec8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1beec0, mutex_guard=0x7f91cda68628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1beec0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1beeb0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1beeb0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cda689a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cda68e98, core=0x7f956917a590, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cda68e98, core=0x7f956917a590) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cda68e98, core=0x7f956917a590) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cda6a610, t=0x7f91cda68e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cda6a5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cda68e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cda69098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cda69220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211ea8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211ea8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211ea0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cda69398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211ea0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cda695b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=110) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cda69a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1304 (Thread 0x7f91cdc9e640 (LWP 2741306) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cdc9a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cdc9a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cdc9a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684512, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bee60, mutex=0x7f91ea1bee68, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bee60, mutex_guard=0x7f91cdc69628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bee60, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bee50) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bee50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cdc699a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cdc69e98, core=0x7f956917a540, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cdc69e98, core=0x7f956917a540) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cdc69e98, core=0x7f956917a540) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cdc6b610, t=0x7f91cdc69e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cdc6b5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cdc69e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cdc6a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cdc6a220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211e28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211e28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211e20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cdc6a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211e20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cdc6a5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=109) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cdc6aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1303 (Thread 0x7f91cde9f640 (LWP 2741305) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cde9b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cde9b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cde9b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684416, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bee00, mutex=0x7f91ea1bee08, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bee00, mutex_guard=0x7f91cde6a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bee00, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bedf0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bedf0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cde6a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cde6ae98, core=0x7f956917a4f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cde6ae98, core=0x7f956917a4f0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cde6ae98, core=0x7f956917a4f0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cde6c610, t=0x7f91cde6ae90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cde6c5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cde6ae90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cde6b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cde6b220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211da8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211da8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211da0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cde6b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211da0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cde6b5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=108) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cde6ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1302 (Thread 0x7f91ce0a0640 (LWP 2741304) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ce09c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ce09c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ce09c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684224, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bed40, mutex=0x7f91ea1bed48, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bed40, mutex_guard=0x7f91ce06b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bed40, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bed30) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bed30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ce06b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ce06be98, core=0x7f956917a450, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ce06be98, core=0x7f956917a450) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ce06be98, core=0x7f956917a450) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ce06d610, t=0x7f91ce06be90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ce06d5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ce06be90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ce06c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ce06c220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211ca8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211ca8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211ca0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ce06c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211ca0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ce06c5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=107) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ce06ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1301 (Thread 0x7f91ce2a1640 (LWP 2741303) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ce29d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ce29d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ce29d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684320, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1beda0, mutex=0x7f91ea1beda8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1beda0, mutex_guard=0x7f91ce26c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1beda0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bed90) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bed90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ce26c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ce26ce98, core=0x7f956917a4a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ce26ce98, core=0x7f956917a4a0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ce26ce98, core=0x7f956917a4a0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ce26e610, t=0x7f91ce26ce90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ce26e5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ce26ce90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ce26d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ce26d220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211d28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211d28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211d20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ce26d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211d20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ce26d5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=106) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ce26da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1300 (Thread 0x7f91ce4a2640 (LWP 2741302) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ce49e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ce49e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ce49e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684128, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bece0, mutex=0x7f91ea1bece8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bece0, mutex_guard=0x7f91ce46d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bece0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1becd0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1becd0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ce46d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ce46de98, core=0x7f956917a400, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ce46de98, core=0x7f956917a400) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ce46de98, core=0x7f956917a400) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ce46f610, t=0x7f91ce46de90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ce46f5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ce46de90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ce46e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ce46e220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211c28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211c28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211c20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ce46e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211c20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ce46e5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=105) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ce46ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1299 (Thread 0x7f91ce6a3640 (LWP 2741301) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ce69f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ce69f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ce69f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674684032, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bec80, mutex=0x7f91ea1bec88, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bec80, mutex_guard=0x7f91ce66e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bec80, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bec70) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bec70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ce66e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ce66ee98, core=0x7f956917a3b0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ce66ee98, core=0x7f956917a3b0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ce66ee98, core=0x7f956917a3b0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ce670610, t=0x7f91ce66ee90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ce6705d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ce66ee90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ce66f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ce66f220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211ba8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211ba8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211ba0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ce66f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211ba0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ce66f5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=104) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ce66fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1298 (Thread 0x7f91ce8a4640 (LWP 2741300) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ce8a0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ce8a0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ce8a0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674683936, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bec20, mutex=0x7f91ea1bec28, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bec20, mutex_guard=0x7f91ce86f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bec20, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bec10) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bec10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ce86f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ce86fe98, core=0x7f956917a360, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ce86fe98, core=0x7f956917a360) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ce86fe98, core=0x7f956917a360) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ce871610, t=0x7f91ce86fe90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ce8715d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ce86fe90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ce870098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ce870220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211b28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211b28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211b20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ce870398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211b20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ce8705b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=103) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ce870a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1297 (Thread 0x7f91ced25640 (LWP 2741299) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ced21750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ced21750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ced21730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674683840, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bebc0, mutex=0x7f91ea1bebc8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bebc0, mutex_guard=0x7f91cecf0628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bebc0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bebb0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bebb0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cecf09a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cecf0e98, core=0x7f956917a310, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cecf0e98, core=0x7f956917a310) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cecf0e98, core=0x7f956917a310) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cecf2610, t=0x7f91cecf0e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cecf25d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cecf0e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cecf1098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cecf1220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211aa8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211aa8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211aa0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cecf1398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211aa0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cecf15b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=102) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cecf1a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1296 (Thread 0x7f91cef26640 (LWP 2741298) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cef22750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cef22750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cef22730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674683744, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1beb60, mutex=0x7f91ea1beb68, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1beb60, mutex_guard=0x7f91ceef1628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1beb60, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1beb50) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1beb50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ceef19a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ceef1e98, core=0x7f956917a2c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ceef1e98, core=0x7f956917a2c0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ceef1e98, core=0x7f956917a2c0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ceef3610, t=0x7f91ceef1e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ceef35d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ceef1e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ceef2098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ceef2220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211a28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211a28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211a20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ceef2398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211a20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ceef25b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=101) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ceef2a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1295 (Thread 0x7f91cf127640 (LWP 2741297) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cf123750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cf123750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cf123730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674683648, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1beb00, mutex=0x7f91ea1beb08, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1beb00, mutex_guard=0x7f91cf0f2628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1beb00, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1beaf0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1beaf0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cf0f29a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cf0f2e98, core=0x7f956917a270, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cf0f2e98, core=0x7f956917a270) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cf0f2e98, core=0x7f956917a270) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cf0f4610, t=0x7f91cf0f2e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cf0f45d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cf0f2e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cf0f3098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cf0f3220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2119a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2119a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2119a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cf0f3398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2119a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cf0f35b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=100) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cf0f3a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1294 (Thread 0x7f91cf328640 (LWP 2741296) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cf324750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cf324750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cf324730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674683552, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1beaa0, mutex=0x7f91ea1beaa8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1beaa0, mutex_guard=0x7f91cf2f3628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1beaa0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bea90) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bea90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cf2f39a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cf2f3e98, core=0x7f956917a220, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cf2f3e98, core=0x7f956917a220) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cf2f3e98, core=0x7f956917a220) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cf2f5610, t=0x7f91cf2f3e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cf2f55d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cf2f3e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cf2f4098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cf2f4220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211928) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211928, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211920, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cf2f4398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211920, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cf2f45b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=99) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cf2f4a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1293 (Thread 0x7f91cf529640 (LWP 2741295) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cf525750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cf525750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cf525730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674683360, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be9e0, mutex=0x7f91ea1be9e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be9e0, mutex_guard=0x7f91cf4f4628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be9e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be9d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be9d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cf4f49a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cf4f4e98, core=0x7f956917a180, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cf4f4e98, core=0x7f956917a180) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cf4f4e98, core=0x7f956917a180) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cf4f6610, t=0x7f91cf4f4e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cf4f65d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cf4f4e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cf4f5098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cf4f5220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211828) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211828, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211820, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cf4f5398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211820, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cf4f55b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=98) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cf4f5a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1292 (Thread 0x7f91d0b2a640 (LWP 2741294) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d0b26750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d0b26750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d0b26730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674683264, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be980, mutex=0x7f91ea1be988, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be980, mutex_guard=0x7f91d0af5628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be980, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be970) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be970, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d0af59a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d0af5e98, core=0x7f956917a130, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d0af5e98, core=0x7f956917a130) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d0af5e98, core=0x7f956917a130) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d0af7610, t=0x7f91d0af5e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d0af75d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d0af5e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d0af6098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d0af6220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2117a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2117a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2117a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d0af6398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2117a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d0af65b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=97) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d0af6a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1291 (Thread 0x7f91d0d2b640 (LWP 2741293) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d0d27750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d0d27750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d0d27730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674683168, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be920, mutex=0x7f91ea1be928, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be920, mutex_guard=0x7f91d0cf6628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be920, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be910) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be910, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d0cf69a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d0cf6e98, core=0x7f956917a0e0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d0cf6e98, core=0x7f956917a0e0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d0cf6e98, core=0x7f956917a0e0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d0cf8610, t=0x7f91d0cf6e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d0cf85d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d0cf6e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d0cf7098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d0cf7220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211728) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211728, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211720, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d0cf7398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211720, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d0cf75b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=96) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d0cf7a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1290 (Thread 0x7f91d0f2c640 (LWP 2741292) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d0f28750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d0f28750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d0f28730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674683072, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be8c0, mutex=0x7f91ea1be8c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be8c0, mutex_guard=0x7f91d0ef7628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be8c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be8b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be8b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d0ef79a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d0ef7e98, core=0x7f956917a090, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d0ef7e98, core=0x7f956917a090) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d0ef7e98, core=0x7f956917a090) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d0ef9610, t=0x7f91d0ef7e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d0ef95d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d0ef7e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d0ef8098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d0ef8220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2116a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2116a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2116a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d0ef8398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2116a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d0ef85b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=95) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d0ef8a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1289 (Thread 0x7f91d112d640 (LWP 2741291) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d1129750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d1129750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d1129730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682976, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be860, mutex=0x7f91ea1be868, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be860, mutex_guard=0x7f91d10f8628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be860, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be850) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be850, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d10f89a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d10f8e98, core=0x7f956917a040, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d10f8e98, core=0x7f956917a040) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d10f8e98, core=0x7f956917a040) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d10fa610, t=0x7f91d10f8e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d10fa5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d10f8e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d10f9098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d10f9220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211628) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211628, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211620, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d10f9398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211620, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d10f95b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=94) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d10f9a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1288 (Thread 0x7f91d132e640 (LWP 2741290) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d132a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d132a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d132a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674683456, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bea40, mutex=0x7f91ea1bea48, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bea40, mutex_guard=0x7f91d12f9628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bea40, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bea30) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bea30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d12f99a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d12f9e98, core=0x7f956917a1d0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d12f9e98, core=0x7f956917a1d0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d12f9e98, core=0x7f956917a1d0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d12fb610, t=0x7f91d12f9e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d12fb5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d12f9e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d12fa098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d12fa220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2118a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2118a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2118a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d12fa398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2118a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d12fa5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=93) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d12faa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1287 (Thread 0x7f91d152f640 (LWP 2741289) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d152b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d152b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d152b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682880, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be800, mutex=0x7f91ea1be808, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be800, mutex_guard=0x7f91d14fa628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be800, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be7f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be7f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d14fa9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d14fae98, core=0x7f9569179ff0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d14fae98, core=0x7f9569179ff0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d14fae98, core=0x7f9569179ff0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d14fc610, t=0x7f91d14fae90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d14fc5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d14fae90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d14fb098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d14fb220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2115a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2115a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2115a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d14fb398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2115a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d14fb5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=92) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d14fba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1286 (Thread 0x7f91d1730640 (LWP 2741288) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d172c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d172c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d172c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682784, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be7a0, mutex=0x7f91ea1be7a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be7a0, mutex_guard=0x7f91d16fb628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be7a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be790) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be790, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d16fb9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d16fbe98, core=0x7f9569179fa0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d16fbe98, core=0x7f9569179fa0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d16fbe98, core=0x7f9569179fa0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d16fd610, t=0x7f91d16fbe90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d16fd5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d16fbe90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d16fc098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d16fc220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211528) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211528, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211520, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d16fc398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211520, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d16fc5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=91) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d16fca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1285 (Thread 0x7f91d1931640 (LWP 2741287) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d192d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d192d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d192d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682688, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be740, mutex=0x7f91ea1be748, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be740, mutex_guard=0x7f91d18fc628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be740, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be730) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be730, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d18fc9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d18fce98, core=0x7f9569179f50, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d18fce98, core=0x7f9569179f50) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d18fce98, core=0x7f9569179f50) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d18fe610, t=0x7f91d18fce90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d18fe5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d18fce90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d18fd098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d18fd220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2114a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2114a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2114a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d18fd398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2114a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d18fd5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=90) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d18fda20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1284 (Thread 0x7f91d1b32640 (LWP 2741286) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d1b2e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d1b2e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d1b2e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be6e0, mutex=0x7f91ea1be6e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be6e0, mutex_guard=0x7f91d1afd628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be6e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be6d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be6d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d1afd9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d1afde98, core=0x7f9569179f00, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d1afde98, core=0x7f9569179f00) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d1afde98, core=0x7f9569179f00) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d1aff610, t=0x7f91d1afde90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d1aff5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d1afde90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d1afe098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d1afe220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211428) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211428, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211420, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d1afe398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211420, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d1afe5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=89) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d1afea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1283 (Thread 0x7f91d1d33640 (LWP 2741285) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d1d2f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d1d2f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d1d2f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682496, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be680, mutex=0x7f91ea1be688, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be680, mutex_guard=0x7f91d1cfe628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be680, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be670) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be670, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d1cfe9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d1cfee98, core=0x7f9569179eb0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d1cfee98, core=0x7f9569179eb0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d1cfee98, core=0x7f9569179eb0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d1d00610, t=0x7f91d1cfee90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d1d005d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d1cfee90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d1cff098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d1cff220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2113a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2113a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2113a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d1cff398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2113a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d1cff5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=88) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d1cffa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1282 (Thread 0x7f91d1f34640 (LWP 2741284) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d1f30750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d1f30750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d1f30730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682400, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be620, mutex=0x7f91ea1be628, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be620, mutex_guard=0x7f91d1eff628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be620, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be610) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be610, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d1eff9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d1effe98, core=0x7f9569179e60, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d1effe98, core=0x7f9569179e60) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d1effe98, core=0x7f9569179e60) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d1f01610, t=0x7f91d1effe90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d1f015d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d1effe90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d1f00098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d1f00220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211328) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211328, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211320, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d1f00398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211320, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d1f005b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=87) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d1f00a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1281 (Thread 0x7f91d2135640 (LWP 2741283) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2131750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2131750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2131730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682304, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be5c0, mutex=0x7f91ea1be5c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be5c0, mutex_guard=0x7f91d2100628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be5c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be5b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be5b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d21009a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2100e98, core=0x7f9569179e10, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2100e98, core=0x7f9569179e10) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2100e98, core=0x7f9569179e10) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2102610, t=0x7f91d2100e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d21025d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2100e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2101098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2101220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2112a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2112a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2112a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2101398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2112a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d21015b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=86) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2101a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1280 (Thread 0x7f91d2336640 (LWP 2741282) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2332750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2332750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2332730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682208, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be560, mutex=0x7f91ea1be568, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be560, mutex_guard=0x7f91d2301628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be560, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be550) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be550, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d23019a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2301e98, core=0x7f9569179dc0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2301e98, core=0x7f9569179dc0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2301e98, core=0x7f9569179dc0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2303610, t=0x7f91d2301e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d23035d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2301e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2302098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2302220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211228) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211228, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211220, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2302398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211220, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d23025b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=85) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2302a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1279 (Thread 0x7f91d2537640 (LWP 2741281) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2533750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2533750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2533730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682112, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be500, mutex=0x7f91ea1be508, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be500, mutex_guard=0x7f91d2502628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be500, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be4f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be4f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d25029a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2502e98, core=0x7f9569179d70, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2502e98, core=0x7f9569179d70) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2502e98, core=0x7f9569179d70) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2504610, t=0x7f91d2502e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d25045d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2502e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2503098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2503220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2111a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2111a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2111a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2503398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2111a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d25035b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=84) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2503a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1278 (Thread 0x7f91d2738640 (LWP 2741280) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2734750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2734750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2734730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674682016, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be4a0, mutex=0x7f91ea1be4a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be4a0, mutex_guard=0x7f91d2703628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be4a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be490) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be490, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d27039a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2703e98, core=0x7f9569179d20, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2703e98, core=0x7f9569179d20) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2703e98, core=0x7f9569179d20) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2705610, t=0x7f91d2703e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d27055d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2703e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2704098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2704220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211128) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211128, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211120, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2704398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211120, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d27045b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=83) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2704a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1277 (Thread 0x7f91d2939640 (LWP 2741279) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2935750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2935750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2935730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674681920, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be440, mutex=0x7f91ea1be448, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be440, mutex_guard=0x7f91d2904628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be440, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be430) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be430, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d29049a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2904e98, core=0x7f9569179cd0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2904e98, core=0x7f9569179cd0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2904e98, core=0x7f9569179cd0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2906610, t=0x7f91d2904e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d29065d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2904e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2905098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2905220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2110a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2110a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2110a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2905398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2110a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d29055b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=82) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2905a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1276 (Thread 0x7f91d2b3a640 (LWP 2741278) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2b36750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2b36750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2b36730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674681728, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be380, mutex=0x7f91ea1be388, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be380, mutex_guard=0x7f91d2b05628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be380, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be370) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be370, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d2b059a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2b05e98, core=0x7f9569179c30, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2b05e98, core=0x7f9569179c30) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2b05e98, core=0x7f9569179c30) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2b07610, t=0x7f91d2b05e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d2b075d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2b05e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2b06098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2b06220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bfa8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bfa8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bfa0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2b06398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bfa0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d2b065b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=81) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2b06a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1275 (Thread 0x7f91d2d3b640 (LWP 2741277) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2d37750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2d37750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2d37730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674681824, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be3e0, mutex=0x7f91ea1be3e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be3e0, mutex_guard=0x7f91d2d06628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be3e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be3d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be3d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d2d069a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2d06e98, core=0x7f9569179c80, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2d06e98, core=0x7f9569179c80) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2d06e98, core=0x7f9569179c80) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2d08610, t=0x7f91d2d06e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d2d085d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2d06e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2d07098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2d07220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211028) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211028, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211020, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2d07398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211020, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d2d075b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=80) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2d07a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1274 (Thread 0x7f91d2f3c640 (LWP 2741276) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2f38750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2f38750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2f38730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674681632, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be320, mutex=0x7f91ea1be328, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be320, mutex_guard=0x7f91d2f07628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be320, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be310) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be310, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d2f079a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2f07e98, core=0x7f9569179be0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2f07e98, core=0x7f9569179be0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2f07e98, core=0x7f9569179be0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2f09610, t=0x7f91d2f07e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d2f095d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2f07e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2f08098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2f08220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bf28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bf28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bf20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2f08398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bf20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d2f085b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=79) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2f08a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1273 (Thread 0x7f91d313d640 (LWP 2741275) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d3139750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d3139750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d3139730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674681536, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be2c0, mutex=0x7f91ea1be2c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be2c0, mutex_guard=0x7f91d3108628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be2c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be2b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be2b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d31089a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d3108e98, core=0x7f9569179b90, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d3108e98, core=0x7f9569179b90) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d3108e98, core=0x7f9569179b90) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d310a610, t=0x7f91d3108e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d310a5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d3108e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d3109098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d3109220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bea8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bea8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bea0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d3109398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bea0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d31095b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=78) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d3109a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1272 (Thread 0x7f91d333e640 (LWP 2741274) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d333a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d333a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d333a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674681440, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be260, mutex=0x7f91ea1be268, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be260, mutex_guard=0x7f91d3309628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be260, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be250) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be250, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d33099a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d3309e98, core=0x7f9569179b40, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d3309e98, core=0x7f9569179b40) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d3309e98, core=0x7f9569179b40) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d330b610, t=0x7f91d3309e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d330b5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d3309e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d330a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d330a220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20be28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20be28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20be20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d330a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20be20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d330a5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=77) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d330aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1271 (Thread 0x7f91d353f640 (LWP 2741273) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d353b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d353b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d353b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674681344, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be200, mutex=0x7f91ea1be208, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be200, mutex_guard=0x7f91d350a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be200, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be1f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be1f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d350a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d350ae98, core=0x7f9569179af0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d350ae98, core=0x7f9569179af0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d350ae98, core=0x7f9569179af0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d350c610, t=0x7f91d350ae90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d350c5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d350ae90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d350b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d350b220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bda8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bda8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bda0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d350b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bda0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d350b5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=76) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d350ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1270 (Thread 0x7f91d3740640 (LWP 2741272) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d373c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d373c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d373c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674681248, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be1a0, mutex=0x7f91ea1be1a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be1a0, mutex_guard=0x7f91d370b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be1a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be190) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be190, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d370b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d370be98, core=0x7f9569179aa0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d370be98, core=0x7f9569179aa0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d370be98, core=0x7f9569179aa0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d370d610, t=0x7f91d370be90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d370d5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d370be90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d370c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d370c220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bd28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bd28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bd20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d370c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bd20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d370c5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=75) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d370ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1269 (Thread 0x7f91d3941640 (LWP 2741271) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d393d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d393d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d393d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674681152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be140, mutex=0x7f91ea1be148, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be140, mutex_guard=0x7f91d390c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be140, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be130) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be130, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d390c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d390ce98, core=0x7f9569179a50, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d390ce98, core=0x7f9569179a50) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d390ce98, core=0x7f9569179a50) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d390e610, t=0x7f91d390ce90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d390e5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d390ce90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d390d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d390d220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bca8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bca8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bca0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d390d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bca0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d390d5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=74) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d390da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1268 (Thread 0x7f91d3b42640 (LWP 2741270) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d3b3e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d3b3e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d3b3e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674680960, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be080, mutex=0x7f91ea1be088, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be080, mutex_guard=0x7f91d3b0d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be080, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be070) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be070, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d3b0d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d3b0de98, core=0x7f95691799b0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d3b0de98, core=0x7f95691799b0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d3b0de98, core=0x7f95691799b0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d3b0f610, t=0x7f91d3b0de90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d3b0f5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d3b0de90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d3b0e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d3b0e220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bba8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bba8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bba0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d3b0e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bba0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d3b0e5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=73) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d3b0ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1267 (Thread 0x7f91d3d43640 (LWP 2741269) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d3d3f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d3d3f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d3d3f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674681056, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be0e0, mutex=0x7f91ea1be0e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be0e0, mutex_guard=0x7f91d3d0e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be0e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be0d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be0d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d3d0e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d3d0ee98, core=0x7f9569179a00, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d3d0ee98, core=0x7f9569179a00) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d3d0ee98, core=0x7f9569179a00) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d3d10610, t=0x7f91d3d0ee90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d3d105d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d3d0ee90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d3d0f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d3d0f220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bc28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bc28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bc20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d3d0f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bc20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d3d0f5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=72) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d3d0fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1266 (Thread 0x7f91d3f44640 (LWP 2741268) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d3f40750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d3f40750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d3f40730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264674680864, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be020, mutex=0x7f91ea1be028, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be020, mutex_guard=0x7f91d3f0f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be020, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be010) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be010, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d3f0f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d3f0fe98, core=0x7f9569179960, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d3f0fe98, core=0x7f9569179960) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d3f0fe98, core=0x7f9569179960) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d3f11610, t=0x7f91d3f0fe90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d3f115d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d3f0fe90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d3f10098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d3f10220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bb28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bb28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bb20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d3f10398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bb20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d3f105b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=71) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d3f10a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1265 (Thread 0x7f91d4145640 (LWP 2741267) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4141750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4141750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4141730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238976, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dfc0, mutex=0x7f91ea05dfc8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dfc0, mutex_guard=0x7f91d4110628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dfc0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dfb0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dfb0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d41109a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4110e98, core=0x7f9569179910, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4110e98, core=0x7f9569179910) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4110e98, core=0x7f9569179910) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4112610, t=0x7f91d4110e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d41125d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4110e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4111098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4111220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20baa8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20baa8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20baa0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4111398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20baa0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d41115b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=70) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4111a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1264 (Thread 0x7f91d4346640 (LWP 2741266) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4342750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4342750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4342730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238784, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05df00, mutex=0x7f91ea05df08, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05df00, mutex_guard=0x7f91d4311628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05df00, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05def0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05def0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d43119a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4311e98, core=0x7f9569179870, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4311e98, core=0x7f9569179870) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4311e98, core=0x7f9569179870) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4313610, t=0x7f91d4311e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d43135d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4311e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4312098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4312220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b9a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b9a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b9a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4312398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b9a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d43125b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=69) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4312a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1263 (Thread 0x7f91d4547640 (LWP 2741265) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4543750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4543750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4543730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238880, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05df60, mutex=0x7f91ea05df68, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05df60, mutex_guard=0x7f91d4512628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05df60, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05df50) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05df50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d45129a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4512e98, core=0x7f95691798c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4512e98, core=0x7f95691798c0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4512e98, core=0x7f95691798c0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4514610, t=0x7f91d4512e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d45145d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4512e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4513098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4513220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20ba28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20ba28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20ba20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4513398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20ba20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d45135b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=68) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4513a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1262 (Thread 0x7f91d4748640 (LWP 2741264) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4744750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4744750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4744730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238688, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dea0, mutex=0x7f91ea05dea8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dea0, mutex_guard=0x7f91d4713628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dea0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05de90) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05de90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d47139a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4713e98, core=0x7f9569179820, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4713e98, core=0x7f9569179820) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4713e98, core=0x7f9569179820) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4715610, t=0x7f91d4713e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d47155d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4713e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4714098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4714220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b928) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b928, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b920, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4714398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b920, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d47145b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=67) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4714a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1261 (Thread 0x7f91d4949640 (LWP 2741263) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4945750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4945750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4945730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05de40, mutex=0x7f91ea05de48, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05de40, mutex_guard=0x7f91d4914628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05de40, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05de30) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05de30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d49149a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4914e98, core=0x7f95691797d0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4914e98, core=0x7f95691797d0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4914e98, core=0x7f95691797d0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4916610, t=0x7f91d4914e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d49165d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4914e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4915098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4915220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b8a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b8a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b8a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4915398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b8a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d49155b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=66) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4915a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1260 (Thread 0x7f91d4b4a640 (LWP 2741262) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4b46750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4b46750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4b46730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238496, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dde0, mutex=0x7f91ea05dde8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dde0, mutex_guard=0x7f91d4b15628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dde0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ddd0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ddd0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d4b159a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4b15e98, core=0x7f9569179780, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4b15e98, core=0x7f9569179780) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4b15e98, core=0x7f9569179780) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4b17610, t=0x7f91d4b15e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d4b175d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4b15e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4b16098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4b16220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b828) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b828, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b820, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4b16398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b820, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d4b165b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=65) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4b16a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1259 (Thread 0x7f91d4d4b640 (LWP 2741261) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4d47750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4d47750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4d47730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238400, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dd80, mutex=0x7f91ea05dd88, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dd80, mutex_guard=0x7f91d4d16628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dd80, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dd70) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dd70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d4d169a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4d16e98, core=0x7f9569179730, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4d16e98, core=0x7f9569179730) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4d16e98, core=0x7f9569179730) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4d18610, t=0x7f91d4d16e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d4d185d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4d16e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4d17098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4d17220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b7a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b7a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b7a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4d17398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b7a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d4d175b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=64) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4d17a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1258 (Thread 0x7f91d4f4c640 (LWP 2741260) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4f48750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4f48750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4f48730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238304, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dd20, mutex=0x7f91ea05dd28, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dd20, mutex_guard=0x7f91d4f17628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dd20, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dd10) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dd10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d4f179a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4f17e98, core=0x7f95691795f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4f17e98, core=0x7f95691795f0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4f17e98, core=0x7f95691795f0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4f19610, t=0x7f91d4f17e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d4f195d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4f17e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4f18098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4f18220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b728) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b728, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b720, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4f18398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b720, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d4f185b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=63) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4f18a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1257 (Thread 0x7f91d514d640 (LWP 2741259) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d5149750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d5149750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d5149730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238208, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dcc0, mutex=0x7f91ea05dcc8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dcc0, mutex_guard=0x7f91d5118628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dcc0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dcb0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dcb0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d51189a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d5118e98, core=0x7f95691795a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d5118e98, core=0x7f95691795a0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d5118e98, core=0x7f95691795a0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d511a610, t=0x7f91d5118e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d511a5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d5118e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d5119098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d5119220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b6a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b6a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b6a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d5119398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b6a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d51195b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=62) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d5119a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1256 (Thread 0x7f91d534e640 (LWP 2741258) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d534a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d534a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d534a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238112, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dc60, mutex=0x7f91ea05dc68, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dc60, mutex_guard=0x7f91d5319628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dc60, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dc50) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dc50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d53199a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d5319e98, core=0x7f9569179550, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d5319e98, core=0x7f9569179550) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d5319e98, core=0x7f9569179550) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d531b610, t=0x7f91d5319e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d531b5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d5319e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d531a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d531a220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b628) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b628, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b620, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d531a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b620, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d531a5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=61) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d531aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1255 (Thread 0x7f91d554f640 (LWP 2741257) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d554b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d554b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d554b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673238016, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dc00, mutex=0x7f91ea05dc08, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dc00, mutex_guard=0x7f91d551a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dc00, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dbf0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dbf0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d551a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d551ae98, core=0x7f9569179500, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d551ae98, core=0x7f9569179500) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d551ae98, core=0x7f9569179500) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d551c610, t=0x7f91d551ae90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d551c5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d551ae90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d551b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d551b220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b5a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b5a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b5a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d551b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b5a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d551b5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=60) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d551ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1254 (Thread 0x7f91d5750640 (LWP 2741256) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d574c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d574c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d574c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673237920, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dba0, mutex=0x7f91ea05dba8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dba0, mutex_guard=0x7f91d571b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dba0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05db90) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05db90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d571b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d571be98, core=0x7f95691794b0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d571be98, core=0x7f95691794b0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d571be98, core=0x7f95691794b0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d571d610, t=0x7f91d571be90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d571d5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d571be90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d571c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d571c220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b528) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b528, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b520, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d571c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b520, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d571c5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=59) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d571ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1253 (Thread 0x7f91d5951640 (LWP 2741255) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d594d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d594d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d594d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673237824, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05db40, mutex=0x7f91ea05db48, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05db40, mutex_guard=0x7f91d591c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05db40, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05db30) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05db30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d591c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d591ce98, core=0x7f9569179460, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d591ce98, core=0x7f9569179460) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d591ce98, core=0x7f9569179460) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d591e610, t=0x7f91d591ce90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d591e5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d591ce90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d591d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d591d220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b4a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b4a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b4a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d591d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b4a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d591d5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=58) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d591da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1252 (Thread 0x7f91d5b52640 (LWP 2741254) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d5b4e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d5b4e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d5b4e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673237728, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dae0, mutex=0x7f91ea05dae8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dae0, mutex_guard=0x7f91d5b1d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dae0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dad0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dad0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d5b1d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d5b1de98, core=0x7f9569179410, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d5b1de98, core=0x7f9569179410) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d5b1de98, core=0x7f9569179410) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d5b1f610, t=0x7f91d5b1de90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d5b1f5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d5b1de90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d5b1e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d5b1e220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b428) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b428, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b420, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d5b1e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b420, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d5b1e5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=57) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d5b1ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1251 (Thread 0x7f91d5d53640 (LWP 2741253) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d5d4f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d5d4f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d5d4f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673237632, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05da80, mutex=0x7f91ea05da88, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05da80, mutex_guard=0x7f91d5d1e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05da80, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05da70) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05da70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d5d1e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d5d1ee98, core=0x7f95691793c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d5d1ee98, core=0x7f95691793c0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d5d1ee98, core=0x7f95691793c0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d5d20610, t=0x7f91d5d1ee90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d5d205d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d5d1ee90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d5d1f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d5d1f220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b3a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b3a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b3a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d5d1f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b3a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d5d1f5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=56) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d5d1fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1250 (Thread 0x7f91d5f54640 (LWP 2741252) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d5f50750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d5f50750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d5f50730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673237536, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05da20, mutex=0x7f91ea05da28, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05da20, mutex_guard=0x7f91d5f1f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05da20, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05da10) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05da10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d5f1f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d5f1fe98, core=0x7f9569179370, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d5f1fe98, core=0x7f9569179370) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d5f1fe98, core=0x7f9569179370) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d5f21610, t=0x7f91d5f1fe90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d5f215d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d5f1fe90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d5f20098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d5f20220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b328) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b328, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b320, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d5f20398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b320, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d5f205b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=55) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d5f20a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1249 (Thread 0x7f91d6155640 (LWP 2741251) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6151750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6151750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6151730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673237440, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d9c0, mutex=0x7f91ea05d9c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d9c0, mutex_guard=0x7f91d6120628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d9c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d9b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d9b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d61209a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6120e98, core=0x7f9569179320, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6120e98, core=0x7f9569179320) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6120e98, core=0x7f9569179320) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6122610, t=0x7f91d6120e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d61225d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6120e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6121098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6121220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b2a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b2a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b2a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6121398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b2a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d61215b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=54) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6121a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1248 (Thread 0x7f91d6356640 (LWP 2741250) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6352750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6352750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6352730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673237248, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d900, mutex=0x7f91ea05d908, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d900, mutex_guard=0x7f91d6321628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d900, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d8f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d8f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d63219a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6321e98, core=0x7f9569179280, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6321e98, core=0x7f9569179280) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6321e98, core=0x7f9569179280) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6323610, t=0x7f91d6321e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d63235d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6321e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6322098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6322220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b1a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b1a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b1a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6322398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b1a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d63225b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=53) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6322a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1247 (Thread 0x7f91d6557640 (LWP 2741249) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6553750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6553750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6553730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673237344, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d960, mutex=0x7f91ea05d968, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d960, mutex_guard=0x7f91d6522628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d960, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d950) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d950, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d65229a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6522e98, core=0x7f95691792d0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6522e98, core=0x7f95691792d0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6522e98, core=0x7f95691792d0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6524610, t=0x7f91d6522e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d65245d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6522e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6523098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6523220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b228) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b228, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b220, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6523398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b220, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d65235b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=52) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6523a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1246 (Thread 0x7f91d6758640 (LWP 2741248) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6754750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6754750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6754730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673237152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d8a0, mutex=0x7f91ea05d8a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d8a0, mutex_guard=0x7f91d6723628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d8a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d890) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d890, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d67239a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6723e98, core=0x7f9569179230, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6723e98, core=0x7f9569179230) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6723e98, core=0x7f9569179230) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6725610, t=0x7f91d6723e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d67255d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6723e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6724098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6724220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b128) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b128, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b120, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6724398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b120, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d67245b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=51) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6724a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1245 (Thread 0x7f91d6959640 (LWP 2741247) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6955750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6955750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6955730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673237056, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d840, mutex=0x7f91ea05d848, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d840, mutex_guard=0x7f91d6924628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d840, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d830) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d830, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d69249a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6924e98, core=0x7f95691791e0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6924e98, core=0x7f95691791e0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6924e98, core=0x7f95691791e0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6926610, t=0x7f91d6924e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d69265d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6924e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6925098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6925220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b0a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b0a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b0a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6925398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b0a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d69255b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=50) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6925a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1244 (Thread 0x7f91d6b5a640 (LWP 2741246) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6b56750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6b56750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6b56730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236960, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d7e0, mutex=0x7f91ea05d7e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d7e0, mutex_guard=0x7f91d6b25628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d7e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d7d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d7d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d6b259a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6b25e98, core=0x7f9569179190, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6b25e98, core=0x7f9569179190) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6b25e98, core=0x7f9569179190) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6b27610, t=0x7f91d6b25e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d6b275d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6b25e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6b26098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6b26220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b028) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b028, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b020, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6b26398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b020, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d6b265b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=49) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6b26a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1243 (Thread 0x7f91d6d5b640 (LWP 2741245) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6d57750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6d57750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6d57730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236864, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d780, mutex=0x7f91ea05d788, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d780, mutex_guard=0x7f91d6d26628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d780, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d770) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d770, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d6d269a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6d26e98, core=0x7f9569179140, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6d26e98, core=0x7f9569179140) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6d26e98, core=0x7f9569179140) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6d28610, t=0x7f91d6d26e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d6d285d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6d26e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6d27098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6d27220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b55a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b55a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b55a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6d27398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b55a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d6d275b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=48) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6d27a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1242 (Thread 0x7f91d6f5c640 (LWP 2741244) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6f58750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6f58750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6f58730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236768, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d720, mutex=0x7f91ea05d728, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d720, mutex_guard=0x7f91d6f27628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d720, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d710) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d710, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d6f279a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6f27e98, core=0x7f95691790f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6f27e98, core=0x7f95691790f0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6f27e98, core=0x7f95691790f0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6f29610, t=0x7f91d6f27e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d6f295d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6f27e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6f28098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6f28220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c9a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c9a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c9a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6f28398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c9a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d6f285b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=47) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6f28a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1241 (Thread 0x7f91d715d640 (LWP 2741243) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d7159750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d7159750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d7159730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236672, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d6c0, mutex=0x7f91ea05d6c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d6c0, mutex_guard=0x7f91d7128628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d6c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d6b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d6b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d71289a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d7128e98, core=0x7f95691790a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d7128e98, core=0x7f95691790a0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d7128e98, core=0x7f95691790a0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d712a610, t=0x7f91d7128e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d712a5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d7128e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d7129098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d7129220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b5528) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b5528, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b5520, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d7129398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b5520, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d71295b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=46) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d7129a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1240 (Thread 0x7f91d735e640 (LWP 2741242) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d735a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d735a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d735a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236576, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d660, mutex=0x7f91ea05d668, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d660, mutex_guard=0x7f91d7329628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d660, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d650) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d650, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d73299a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d7329e98, core=0x7f9569179050, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d7329e98, core=0x7f9569179050) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d7329e98, core=0x7f9569179050) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d732b610, t=0x7f91d7329e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d732b5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d7329e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d732a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d732a220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b54a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b54a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b54a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d732a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b54a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d732a5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=45) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d732aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1239 (Thread 0x7f91d755f640 (LWP 2741241) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d755b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d755b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d755b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236480, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d600, mutex=0x7f91ea05d608, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d600, mutex_guard=0x7f91d752a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d600, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d5f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d5f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d752a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d752ae98, core=0x7f9569179000, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d752ae98, core=0x7f9569179000) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d752ae98, core=0x7f9569179000) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d752c610, t=0x7f91d752ae90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d752c5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d752ae90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d752b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d752b220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b5428) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b5428, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b5420, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d752b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b5420, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d752b5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=44) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d752ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1238 (Thread 0x7f91d7760640 (LWP 2741240) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d775c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d775c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d775c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236384, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d5a0, mutex=0x7f91ea05d5a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d5a0, mutex_guard=0x7f91d772b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d5a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d590) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d590, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d772b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d772be98, core=0x7f956a845ec0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d772be98, core=0x7f956a845ec0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d772be98, core=0x7f956a845ec0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d772d610, t=0x7f91d772be90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d772d5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d772be90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d772c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d772c220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b53a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b53a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b53a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d772c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b53a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d772c5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=43) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d772ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1237 (Thread 0x7f91d7961640 (LWP 2741239) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d795d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d795d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d795d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236192, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d4e0, mutex=0x7f91ea05d4e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d4e0, mutex_guard=0x7f91d792c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d4e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d4d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d4d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d792c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d792ce98, core=0x7f956a845d80, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d792ce98, core=0x7f956a845d80) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d792ce98, core=0x7f956a845d80) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d792e610, t=0x7f91d792ce90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d792e5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d792ce90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d792d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d792d220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615948a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615948a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615948a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d792d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615948a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d792d5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=42) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d792da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1236 (Thread 0x7f91d7b62640 (LWP 2741238) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d7b5e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d7b5e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d7b5e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236288, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d540, mutex=0x7f91ea05d548, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d540, mutex_guard=0x7f91d7b2d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d540, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d530) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d530, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d7b2d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d7b2de98, core=0x7f956a845dd0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d7b2de98, core=0x7f956a845dd0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d7b2de98, core=0x7f956a845dd0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d7b2f610, t=0x7f91d7b2de90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d7b2f5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d7b2de90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d7b2e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d7b2e220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b5328) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b5328, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b5320, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d7b2e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b5320, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d7b2e5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=41) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d7b2ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1235 (Thread 0x7f91d7d63640 (LWP 2741237) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d7d5f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d7d5f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d7d5f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236096, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d480, mutex=0x7f91ea05d488, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d480, mutex_guard=0x7f91d7d2e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d480, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d470) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d470, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d7d2e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d7d2ee98, core=0x7f956a845d30, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d7d2ee98, core=0x7f956a845d30) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d7d2ee98, core=0x7f956a845d30) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d7d30610, t=0x7f91d7d2ee90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d7d305d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d7d2ee90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d7d2f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d7d2f220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594828) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594828, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594820, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d7d2f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594820, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d7d2f5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=40) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d7d2fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1234 (Thread 0x7f91d7f64640 (LWP 2741236) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d7f60750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d7f60750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d7f60730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673236000, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d420, mutex=0x7f91ea05d428, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d420, mutex_guard=0x7f91d7f2f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d420, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d410) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d410, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d7f2f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d7f2fe98, core=0x7f956a845bf0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d7f2fe98, core=0x7f956a845bf0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d7f2fe98, core=0x7f956a845bf0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d7f31610, t=0x7f91d7f2fe90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d7f315d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d7f2fe90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d7f30098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d7f30220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615947a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615947a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615947a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d7f30398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615947a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d7f305b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=39) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d7f30a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1233 (Thread 0x7f91d8165640 (LWP 2741235) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8161750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8161750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8161730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673235904, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d3c0, mutex=0x7f91ea05d3c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d3c0, mutex_guard=0x7f91d8130628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d3c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d3b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d3b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d81309a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8130e98, core=0x7f956a845b00, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8130e98, core=0x7f956a845b00) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8130e98, core=0x7f956a845b00) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8132610, t=0x7f91d8130e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d81325d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8130e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8131098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8131220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594728) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594728, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594720, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8131398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594720, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d81315b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=38) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8131a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1232 (Thread 0x7f91d8366640 (LWP 2741234) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8362750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8362750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8362730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673235808, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d360, mutex=0x7f91ea05d368, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d360, mutex_guard=0x7f91d8331628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d360, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d350) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d350, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d83319a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8331e98, core=0x7f956a845a60, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8331e98, core=0x7f956a845a60) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8331e98, core=0x7f956a845a60) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8333610, t=0x7f91d8331e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d83335d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8331e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8332098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8332220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615946a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615946a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615946a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8332398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615946a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d83325b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=37) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8332a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1231 (Thread 0x7f91d8567640 (LWP 2741233) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8563750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8563750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8563730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673235712, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d300, mutex=0x7f91ea05d308, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d300, mutex_guard=0x7f91d8532628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d300, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d2f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d2f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d85329a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8532e98, core=0x7f956a845a10, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8532e98, core=0x7f956a845a10) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8532e98, core=0x7f956a845a10) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8534610, t=0x7f91d8532e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d85345d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8532e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8533098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8533220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594628) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594628, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594620, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8533398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594620, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d85335b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=36) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8533a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1230 (Thread 0x7f91d8768640 (LWP 2741232) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8764750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8764750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8764730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673235616, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d2a0, mutex=0x7f91ea05d2a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d2a0, mutex_guard=0x7f91d8733628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d2a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d290) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d290, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d87339a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8733e98, core=0x7f956a8459c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8733e98, core=0x7f956a8459c0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8733e98, core=0x7f956a8459c0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8735610, t=0x7f91d8733e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d87355d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8733e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8734098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8734220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615945a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615945a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615945a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8734398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615945a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d87345b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=35) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8734a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1229 (Thread 0x7f91d8969640 (LWP 2741231) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8965750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8965750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8965730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673235520, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d240, mutex=0x7f91ea05d248, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d240, mutex_guard=0x7f91d8934628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d240, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d230) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d230, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d89349a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8934e98, core=0x7f956a845970, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8934e98, core=0x7f956a845970) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8934e98, core=0x7f956a845970) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8936610, t=0x7f91d8934e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d89365d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8934e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8935098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8935220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594528) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594528, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594520, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8935398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594520, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d89355b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=34) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8935a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1228 (Thread 0x7f91d8b6a640 (LWP 2741230) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8b66750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8b66750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8b66730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673235424, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d1e0, mutex=0x7f91ea05d1e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d1e0, mutex_guard=0x7f91d8b35628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d1e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d1d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d1d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d8b359a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8b35e98, core=0x7f956a845920, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8b35e98, core=0x7f956a845920) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8b35e98, core=0x7f956a845920) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8b37610, t=0x7f91d8b35e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d8b375d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8b35e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8b36098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8b36220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615944a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615944a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615944a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8b36398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615944a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d8b365b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=33) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8b36a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1227 (Thread 0x7f91d8d6b640 (LWP 2741229) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8d67750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8d67750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8d67730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673235328, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d180, mutex=0x7f91ea05d188, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d180, mutex_guard=0x7f91d8d36628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d180, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d170) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d170, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d8d369a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8d36e98, core=0x7f956a8458d0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8d36e98, core=0x7f956a8458d0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8d36e98, core=0x7f956a8458d0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8d38610, t=0x7f91d8d36e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d8d385d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8d36e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8d37098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8d37220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594428) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594428, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594420, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8d37398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594420, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d8d375b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=32) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8d37a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1226 (Thread 0x7f91d8f6c640 (LWP 2741228) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8f68750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8f68750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8f68730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673235232, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d120, mutex=0x7f91ea05d128, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d120, mutex_guard=0x7f91d8f37628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d120, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d110) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d110, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d8f379a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8f37e98, core=0x7f956a845880, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8f37e98, core=0x7f956a845880) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8f37e98, core=0x7f956a845880) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8f39610, t=0x7f91d8f37e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d8f395d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8f37e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8f38098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8f38220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615943a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615943a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615943a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8f38398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615943a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d8f385b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=31) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8f38a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1225 (Thread 0x7f91d916d640 (LWP 2741227) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d9169750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d9169750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d9169730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673235136, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d0c0, mutex=0x7f91ea05d0c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d0c0, mutex_guard=0x7f91d9138628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d0c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d0b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d0b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d91389a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d9138e98, core=0x7f956a845830, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d9138e98, core=0x7f956a845830) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d9138e98, core=0x7f956a845830) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d913a610, t=0x7f91d9138e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d913a5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d9138e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d9139098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d9139220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594328) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594328, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594320, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d9139398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594320, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d91395b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=30) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d9139a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1224 (Thread 0x7f91d936e640 (LWP 2741226) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d936a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d936a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d936a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673235040, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d060, mutex=0x7f91ea05d068, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d060, mutex_guard=0x7f91d9339628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d060, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d050) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d050, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d93399a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d9339e98, core=0x7f956a8457e0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d9339e98, core=0x7f956a8457e0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d9339e98, core=0x7f956a8457e0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d933b610, t=0x7f91d9339e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d933b5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d9339e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d933a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d933a220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615942a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615942a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615942a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d933a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615942a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d933a5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=29) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d933aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1223 (Thread 0x7f91d956f640 (LWP 2741225) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d956b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d956b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d956b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673234944, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d000, mutex=0x7f91ea05d008, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d000, mutex_guard=0x7f91d953a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d000, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cff0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cff0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d953a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d953ae98, core=0x7f956a845790, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d953ae98, core=0x7f956a845790) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d953ae98, core=0x7f956a845790) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d953c610, t=0x7f91d953ae90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d953c5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d953ae90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d953b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d953b220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507928) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507928, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507920, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d953b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507920, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d953b5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=28) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d953ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1222 (Thread 0x7f91d9770640 (LWP 2741224) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d976c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d976c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d976c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673234848, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cfa0, mutex=0x7f91ea05cfa8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cfa0, mutex_guard=0x7f91d973b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cfa0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cf90) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cf90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d973b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d973be98, core=0x7f956a845740, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d973be98, core=0x7f956a845740) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d973be98, core=0x7f956a845740) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d973d610, t=0x7f91d973be90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d973d5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d973be90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d973c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d973c220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615078a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615078a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615078a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d973c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615078a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d973c5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=27) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d973ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1221 (Thread 0x7f91d9971640 (LWP 2741223) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d996d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d996d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d996d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673234752, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cf40, mutex=0x7f91ea05cf48, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cf40, mutex_guard=0x7f91d993c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cf40, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cf30) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cf30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d993c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d993ce98, core=0x7f956a8456a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d993ce98, core=0x7f956a8456a0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d993ce98, core=0x7f956a8456a0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d993e610, t=0x7f91d993ce90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d993e5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d993ce90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d993d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d993d220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507828) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507828, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507820, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d993d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507820, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d993d5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=26) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d993da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1220 (Thread 0x7f91d9b72640 (LWP 2741222) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d9b6e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d9b6e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d9b6e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673234656, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cee0, mutex=0x7f91ea05cee8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cee0, mutex_guard=0x7f91d9b3d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cee0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ced0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ced0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d9b3d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d9b3de98, core=0x7f956a8452e0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d9b3de98, core=0x7f956a8452e0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d9b3de98, core=0x7f956a8452e0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d9b3f610, t=0x7f91d9b3de90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d9b3f5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d9b3de90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d9b3e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d9b3e220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615077a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615077a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615077a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d9b3e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615077a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d9b3e5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=25) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d9b3ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1219 (Thread 0x7f91d9d73640 (LWP 2741221) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d9d6f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d9d6f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d9d6f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673234560, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05ce80, mutex=0x7f91ea05ce88, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05ce80, mutex_guard=0x7f91d9d3e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05ce80, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ce70) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ce70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d9d3e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d9d3ee98, core=0x7f956a845290, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d9d3ee98, core=0x7f956a845290) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d9d3ee98, core=0x7f956a845290) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d9d40610, t=0x7f91d9d3ee90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d9d405d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d9d3ee90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d9d3f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d9d3f220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507728) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507728, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507720, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d9d3f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507720, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d9d3f5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=24) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d9d3fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1218 (Thread 0x7f91d9f74640 (LWP 2741220) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d9f70750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d9f70750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d9f70730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673234464, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05ce20, mutex=0x7f91ea05ce28, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05ce20, mutex_guard=0x7f91d9f3f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05ce20, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ce10) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ce10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d9f3f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d9f3fe98, core=0x7f956a845240, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d9f3fe98, core=0x7f956a845240) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d9f3fe98, core=0x7f956a845240) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d9f41610, t=0x7f91d9f3fe90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d9f415d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d9f3fe90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d9f40098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d9f40220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615076a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615076a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615076a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d9f40398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615076a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d9f405b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=23) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d9f40a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1217 (Thread 0x7f91da175640 (LWP 2741219) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91da171750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91da171750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91da171730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673234368, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cdc0, mutex=0x7f91ea05cdc8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cdc0, mutex_guard=0x7f91da140628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cdc0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cdb0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cdb0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91da1409a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91da140e98, core=0x7f956a8451f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91da140e98, core=0x7f956a8451f0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91da140e98, core=0x7f956a8451f0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91da142610, t=0x7f91da140e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91da1425d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91da140e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91da141098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91da141220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507628) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507628, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507620, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91da141398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507620, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91da1415b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=22) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91da141a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1216 (Thread 0x7f91da376640 (LWP 2741218) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91da372750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91da372750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91da372730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673234272, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cd60, mutex=0x7f91ea05cd68, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cd60, mutex_guard=0x7f91da341628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cd60, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cd50) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cd50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91da3419a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91da341e98, core=0x7f956a844f70, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91da341e98, core=0x7f956a844f70) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91da341e98, core=0x7f956a844f70) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91da343610, t=0x7f91da341e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91da3435d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91da341e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91da342098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91da342220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615075a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615075a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615075a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91da342398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615075a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91da3425b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=21) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91da342a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1215 (Thread 0x7f91da577640 (LWP 2741217) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91da573750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91da573750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91da573730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673234176, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cd00, mutex=0x7f91ea05cd08, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cd00, mutex_guard=0x7f91da542628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cd00, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ccf0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ccf0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91da5429a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91da542e98, core=0x7f956a8437b0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91da542e98, core=0x7f956a8437b0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91da542e98, core=0x7f956a8437b0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91da544610, t=0x7f91da542e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91da5445d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91da542e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91da543098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91da543220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507528) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507528, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507520, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91da543398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507520, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91da5435b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=20) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91da543a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1214 (Thread 0x7f91da778640 (LWP 2741216) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91da774750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91da774750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91da774730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673234080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cca0, mutex=0x7f91ea05cca8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cca0, mutex_guard=0x7f91da743628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cca0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cc90) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cc90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91da7439a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91da743e98, core=0x7f956a843760, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91da743e98, core=0x7f956a843760) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91da743e98, core=0x7f956a843760) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91da745610, t=0x7f91da743e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91da7455d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91da743e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91da744098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91da744220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615074a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615074a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615074a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91da744398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615074a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91da7445b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=19) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91da744a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1213 (Thread 0x7f91da979640 (LWP 2741215) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91da975750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91da975750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91da975730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233984, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cc40, mutex=0x7f91ea05cc48, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cc40, mutex_guard=0x7f91da944628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cc40, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cc30) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cc30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91da9449a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91da944e98, core=0x7f956a8433f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91da944e98, core=0x7f956a8433f0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91da944e98, core=0x7f956a8433f0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91da946610, t=0x7f91da944e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91da9465d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91da944e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91da945098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91da945220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507428) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507428, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507420, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91da945398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507420, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91da9455b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=18) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91da945a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1212 (Thread 0x7f91dab7a640 (LWP 2741214) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dab76750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dab76750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dab76730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233888, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cbe0, mutex=0x7f91ea05cbe8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cbe0, mutex_guard=0x7f91dab45628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cbe0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cbd0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cbd0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dab459a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dab45e98, core=0x7f956a8433a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dab45e98, core=0x7f956a8433a0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dab45e98, core=0x7f956a8433a0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dab47610, t=0x7f91dab45e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dab475d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dab45e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dab46098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dab46220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615073a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615073a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615073a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dab46398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615073a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dab465b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=17) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dab46a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1211 (Thread 0x7f91dad7b640 (LWP 2741213) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dad77750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dad77750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dad77730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233792, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cb80, mutex=0x7f91ea05cb88, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cb80, mutex_guard=0x7f91dad46628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cb80, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cb70) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cb70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dad469a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dad46e98, core=0x7f956a843350, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dad46e98, core=0x7f956a843350) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dad46e98, core=0x7f956a843350) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dad48610, t=0x7f91dad46e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dad485d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dad46e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dad47098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dad47220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507328) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507328, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507320, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dad47398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507320, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dad475b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=16) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dad47a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1210 (Thread 0x7f91daf7c640 (LWP 2741212) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91daf78750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91daf78750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91daf78730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233696, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cb20, mutex=0x7f91ea05cb28, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cb20, mutex_guard=0x7f91daf47628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cb20, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cb10) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cb10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91daf479a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91daf47e98, core=0x7f956a843210, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91daf47e98, core=0x7f956a843210) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91daf47e98, core=0x7f956a843210) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91daf49610, t=0x7f91daf47e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91daf495d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91daf47e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91daf48098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91daf48220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615072a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615072a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615072a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91daf48398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615072a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91daf485b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=15) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91daf48a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1209 (Thread 0x7f91db17d640 (LWP 2741211) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91db179750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91db179750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91db179730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233600, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cac0, mutex=0x7f91ea05cac8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cac0, mutex_guard=0x7f91db148628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cac0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cab0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cab0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91db1489a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91db148e98, core=0x7f956a8431c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91db148e98, core=0x7f956a8431c0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91db148e98, core=0x7f956a8431c0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91db14a610, t=0x7f91db148e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91db14a5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91db148e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91db149098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91db149220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507228) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507228, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507220, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91db149398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507220, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91db1495b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=14) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91db149a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1208 (Thread 0x7f91db37e640 (LWP 2741210) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91db37a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91db37a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91db37a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233504, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05ca60, mutex=0x7f91ea05ca68, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05ca60, mutex_guard=0x7f91db349628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05ca60, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ca50) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ca50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91db3499a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91db349e98, core=0x7f956a843170, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91db349e98, core=0x7f956a843170) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91db349e98, core=0x7f956a843170) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91db34b610, t=0x7f91db349e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91db34b5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91db349e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91db34a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91db34a220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615071a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615071a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615071a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91db34a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615071a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91db34a5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=13) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91db34aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1207 (Thread 0x7f91db57f640 (LWP 2741209) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91db57b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91db57b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91db57b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233408, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05ca00, mutex=0x7f91ea05ca08, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05ca00, mutex_guard=0x7f91db54a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05ca00, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c9f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c9f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91db54a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91db54ae98, core=0x7f956a842fe0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91db54ae98, core=0x7f956a842fe0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91db54ae98, core=0x7f956a842fe0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91db54c610, t=0x7f91db54ae90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91db54c5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91db54ae90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91db54b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91db54b220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507128) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507128, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507120, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91db54b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507120, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91db54b5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=12) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91db54ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1206 (Thread 0x7f91db780640 (LWP 2741208) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91db77c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91db77c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91db77c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233312, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c9a0, mutex=0x7f91ea05c9a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c9a0, mutex_guard=0x7f91db74b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c9a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c990) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c990, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91db74b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91db74be98, core=0x7f956a842f90, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91db74be98, core=0x7f956a842f90) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91db74be98, core=0x7f956a842f90) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91db74d610, t=0x7f91db74be90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91db74d5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91db74be90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91db74c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91db74c220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615070a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615070a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615070a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91db74c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615070a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91db74c5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=11) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91db74ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1205 (Thread 0x7f91db981640 (LWP 2741207) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91db97d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91db97d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91db97d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233216, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c940, mutex=0x7f91ea05c948, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c940, mutex_guard=0x7f91db94c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c940, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c930) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c930, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91db94c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91db94ce98, core=0x7f956a842ea0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91db94ce98, core=0x7f956a842ea0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91db94ce98, core=0x7f956a842ea0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91db94e610, t=0x7f91db94ce90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91db94e5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91db94ce90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91db94d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91db94d220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507028) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507028, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507020, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91db94d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507020, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91db94d5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=10) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91db94da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1204 (Thread 0x7f91dbb82640 (LWP 2741206) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dbb7e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dbb7e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dbb7e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c8e0, mutex=0x7f91ea05c8e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c8e0, mutex_guard=0x7f91dbb4d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c8e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c8d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c8d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dbb4d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dbb4de98, core=0x7f956a842e50, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dbb4de98, core=0x7f956a842e50) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dbb4de98, core=0x7f956a842e50) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dbb4f610, t=0x7f91dbb4de90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dbb4f5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dbb4de90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dbb4e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dbb4e220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919cfa8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919cfa8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919cfa0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dbb4e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919cfa0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dbb4e5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=9) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dbb4ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1203 (Thread 0x7f91dbd83640 (LWP 2741205) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dbd7f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dbd7f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dbd7f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673233024, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c880, mutex=0x7f91ea05c888, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c880, mutex_guard=0x7f91dbd4e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c880, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c870) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c870, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dbd4e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dbd4ee98, core=0x7f956a842e00, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dbd4ee98, core=0x7f956a842e00) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dbd4ee98, core=0x7f956a842e00) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dbd50610, t=0x7f91dbd4ee90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dbd505d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dbd4ee90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dbd4f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dbd4f220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919cf28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919cf28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919cf20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dbd4f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919cf20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dbd4f5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=8) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dbd4fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1202 (Thread 0x7f91dbf84640 (LWP 2741204) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dbf80750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dbf80750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dbf80730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673232928, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c820, mutex=0x7f91ea05c828, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c820, mutex_guard=0x7f91dbf4f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c820, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c810) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c810, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dbf4f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dbf4fe98, core=0x7f956a842db0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dbf4fe98, core=0x7f956a842db0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dbf4fe98, core=0x7f956a842db0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dbf51610, t=0x7f91dbf4fe90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dbf515d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dbf4fe90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dbf50098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dbf50220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919cea8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919cea8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919cea0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dbf50398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919cea0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dbf505b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=7) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dbf50a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1201 (Thread 0x7f91dc185640 (LWP 2741203) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dc181750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dc181750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dc181730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673232832, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c7c0, mutex=0x7f91ea05c7c8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c7c0, mutex_guard=0x7f91dc150628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c7c0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c7b0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c7b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dc1509a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dc150e98, core=0x7f956a842d60, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dc150e98, core=0x7f956a842d60) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dc150e98, core=0x7f956a842d60) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dc152610, t=0x7f91dc150e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dc1525d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dc150e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dc151098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dc151220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919ce28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919ce28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919ce20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dc151398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919ce20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dc1515b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=6) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dc151a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1200 (Thread 0x7f91dc386640 (LWP 2741202) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dc382750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dc382750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dc382730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673232736, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c760, mutex=0x7f91ea05c768, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c760, mutex_guard=0x7f91dc351628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c760, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c750) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c750, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dc3519a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dc351e98, core=0x7f956a842d10, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dc351e98, core=0x7f956a842d10) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dc351e98, core=0x7f956a842d10) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dc353610, t=0x7f91dc351e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dc3535d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dc351e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dc352098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dc352220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919ca28) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919ca28, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919ca20, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dc352398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919ca20, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dc3525b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=5) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dc352a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1199 (Thread 0x7f91dc587640 (LWP 2741201) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dc583750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dc583750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dc583730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673232640, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c700, mutex=0x7f91ea05c708, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c700, mutex_guard=0x7f91dc552628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c700, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c6f0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c6f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dc5529a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dc552e98, core=0x7f956a842cc0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dc552e98, core=0x7f956a842cc0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dc552e98, core=0x7f956a842cc0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dc554610, t=0x7f91dc552e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dc5545d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dc552e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dc553098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dc553220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c928) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c928, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c920, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dc553398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c920, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dc5535b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=4) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dc553a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1198 (Thread 0x7f91dc788640 (LWP 2741200) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dc784750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dc784750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dc784730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673232544, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c6a0, mutex=0x7f91ea05c6a8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c6a0, mutex_guard=0x7f91dc753628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c6a0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c690) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c690, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dc7539a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dc753e98, core=0x7f956a842c70, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dc753e98, core=0x7f956a842c70) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dc753e98, core=0x7f956a842c70) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dc755610, t=0x7f91dc753e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dc7555d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dc753e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dc754098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dc754220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c8a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c8a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c8a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dc754398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c8a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dc7545b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=3) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dc754a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1197 (Thread 0x7f91dc989640 (LWP 2741199) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dc985750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dc985750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dc985730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673232448, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c640, mutex=0x7f91ea05c648, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c640, mutex_guard=0x7f91dc954628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c640, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c630) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c630, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dc9549a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dc954e98, core=0x7f956a842c20, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dc954e98, core=0x7f956a842c20) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dc954e98, core=0x7f956a842c20) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dc956610, t=0x7f91dc954e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dc9565d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dc954e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dc955098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dc955220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c828) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c828, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c820, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dc955398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c820, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dc9555b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=2) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dc955a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1196 (Thread 0x7f91dcb8a640 (LWP 2741198) "impwkr-v1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dcb86750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dcb86750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dcb86730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673232352, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c5e0, mutex=0x7f91ea05c5e8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c5e0, mutex_guard=0x7f91dcb55628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c5e0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c5d0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c5d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dcb559a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dcb55e98, core=0x7f956a842bd0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dcb55e98, core=0x7f956a842bd0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dcb55e98, core=0x7f956a842bd0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dcb57610, t=0x7f91dcb55e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dcb575d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dcb55e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dcb56098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dcb56220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c7a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c7a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c7a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dcb56398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c7a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dcb565b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=1) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dcb56a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1195 (Thread 0x7f91dcd8b640 (LWP 2741197) "impwkr-v1"): +#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 +#1 0x00007f9583185778 in mio::sys::unix::selector::epoll::Selector::select (self=0x7f956919c6b8, events=0x7f956919c698, timeout=...) at src/sys/unix/selector/epoll.rs:97 +#2 0x00007f9583184fd1 in mio::poll::Poll::poll (self=0x7f956919c6b0, events=0x7f956919c698, timeout=...) at src/poll.rs:426 +#3 0x00007f9583167b7c in tokio::runtime::io::driver::Driver::turn (self=0x7f956919c698, handle=0x7f91ea093518, max_wait=...) at src/runtime/io/driver.rs:149 +#4 0x00007f95831677ee in tokio::runtime::io::driver::Driver::park (self=0x7f956919c698, rt_handle=0x7f91ea093518) at src/runtime/io/driver.rs:122 +#5 0x00007f958316a887 in tokio::runtime::signal::Driver::park (self=0x7f956919c698, handle=0x7f91ea093518) at src/runtime/signal/mod.rs:92 +#6 0x00007f9583152787 in tokio::runtime::process::Driver::park (self=0x7f956919c698, handle=0x7f91ea093518) at src/runtime/process.rs:32 +#7 0x00007f95830f1aca in tokio::runtime::driver::IoStack::park (self=0x7f956919c698, handle=0x7f91ea093518) at src/runtime/driver.rs:175 +#8 0x00007f958316a02e in tokio::runtime::time::Driver::park_internal (self=0x7f956919c698, rt_handle=0x7f91ea093518, limit=...) at src/runtime/time/mod.rs:235 +#9 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f956919c698, handle=0x7f91ea093518) at src/runtime/time/mod.rs:165 +#10 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f956919c690, handle=0x7f91ea093518) at src/runtime/driver.rs:332 +#11 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f956919c690, handle=0x7f91ea093518) at src/runtime/driver.rs:71 +#12 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f91ea05c570, driver=0x7f956919c690, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:184 +#13 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c570, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:117 +#14 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dcd569a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 +#15 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dcd56e98, core=0x7f956a842b80, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#16 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dcd56e98, core=0x7f956a842b80) at src/runtime/scheduler/multi_thread/worker.rs:702 +#17 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dcd56e98, core=0x7f956a842b80) at src/runtime/scheduler/multi_thread/worker.rs:553 +#18 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#19 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dcd58610, t=0x7f91dcd56e90, f=...) at src/runtime/context/scoped.rs:40 +#20 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dcd585d8) at src/runtime/context.rs:180 +#21 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#22 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#23 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dcd56e90, f=...) at src/runtime/context.rs:180 +#24 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#25 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dcd57098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#26 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#27 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#28 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dcd57220) at src/runtime/blocking/task.rs:42 +#29 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c728) at src/runtime/task/core.rs:328 +#30 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c728, f=...) at src/loom/std/unsafe_cell.rs:16 +#31 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c720, cx=...) at src/runtime/task/core.rs:317 +#32 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#33 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#34 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dcd57398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#35 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#36 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#37 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#38 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c720, cx=...) at src/runtime/task/harness.rs:473 +#39 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dcd575b0) at src/runtime/task/harness.rs:208 +#40 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#41 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#42 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#43 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#44 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#45 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 +#46 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#47 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#48 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#49 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#50 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dcd57a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#51 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#52 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#53 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#54 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#55 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#56 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#57 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#58 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#59 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#60 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1194 (Thread 0x7f91dcf8c640 (LWP 2741196) "impwkr-v0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dcf88750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dcf88750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dcf88730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673231536, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c2b0, mutex=0x7f91ea05c2b8, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c2b0, mutex_guard=0x7f91dcf57628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c2b0, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c2a0) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c2a0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dcf579a0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dcf57e98, core=0x7f956a842130, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dcf57e98, core=0x7f956a842130) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dcf57e98, core=0x7f956a842130) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dcf59610, t=0x7f91dcf57e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dcf595d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dcf57e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dcf58098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dcf58220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c5a8) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c5a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c5a0, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dcf58398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c5a0, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dcf585b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de590, worker_thread_id=3) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dcf58a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1193 (Thread 0x7f91dd18d640 (LWP 2741195) "impwkr-v0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dd189750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dd189750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dd189730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673231440, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c250, mutex=0x7f91ea05c258, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c250, mutex_guard=0x7f91dd158628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c250, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c240) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c240, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dd1589a0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dd158e98, core=0x7f956a842090, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dd158e98, core=0x7f956a842090) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dd158e98, core=0x7f956a842090) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dd15a610, t=0x7f91dd158e90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dd15a5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dd158e90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dd159098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dd159220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c528) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c528, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c520, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dd159398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c520, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dd1595b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de590, worker_thread_id=2) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dd159a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1192 (Thread 0x7f91dd38e640 (LWP 2741194) "impwkr-v0"): +#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 +#1 0x00007f9583185778 in mio::sys::unix::selector::epoll::Selector::select (self=0x7f956919c3b8, events=0x7f956919c398, timeout=...) at src/sys/unix/selector/epoll.rs:97 +#2 0x00007f9583184fd1 in mio::poll::Poll::poll (self=0x7f956919c3b0, events=0x7f956919c398, timeout=...) at src/poll.rs:426 +#3 0x00007f9583167b7c in tokio::runtime::io::driver::Driver::turn (self=0x7f956919c398, handle=0x7f91ea093318, max_wait=...) at src/runtime/io/driver.rs:149 +#4 0x00007f958316786a in tokio::runtime::io::driver::Driver::park_timeout (self=0x7f956919c398, rt_handle=0x7f91ea093318, duration=...) at src/runtime/io/driver.rs:127 +#5 0x00007f958316a8c0 in tokio::runtime::signal::Driver::park_timeout (self=0x7f956919c398, handle=0x7f91ea093318, duration=...) at src/runtime/signal/mod.rs:97 +#6 0x00007f95831527c0 in tokio::runtime::process::Driver::park_timeout (self=0x7f956919c398, handle=0x7f91ea093318, duration=...) at src/runtime/process.rs:37 +#7 0x00007f95830f1b55 in tokio::runtime::driver::IoStack::park_timeout (self=0x7f956919c398, handle=0x7f91ea093318, duration=...) at src/runtime/driver.rs:182 +#8 0x00007f958316a49c in tokio::runtime::time::Driver::park_thread_timeout (self=0x7f956919c398, rt_handle=0x7f91ea093318, duration=...) at src/runtime/time/mod.rs:270 +#9 0x00007f958316a1f5 in tokio::runtime::time::Driver::park_internal (self=0x7f956919c398, rt_handle=0x7f91ea093318, limit=...) at src/runtime/time/mod.rs:226 +#10 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f956919c398, handle=0x7f91ea093318) at src/runtime/time/mod.rs:165 +#11 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f956919c390, handle=0x7f91ea093318) at src/runtime/driver.rs:332 +#12 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f956919c390, handle=0x7f91ea093318) at src/runtime/driver.rs:71 +#13 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f91ea05c1e0, driver=0x7f956919c390, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:184 +#14 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c1e0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:117 +#15 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dd3599a0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:67 +#16 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dd359e98, core=0x7f956a842040, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#17 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dd359e98, core=0x7f956a842040) at src/runtime/scheduler/multi_thread/worker.rs:702 +#18 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dd359e98, core=0x7f956a842040) at src/runtime/scheduler/multi_thread/worker.rs:553 +#19 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#20 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dd35b610, t=0x7f91dd359e90, f=...) at src/runtime/context/scoped.rs:40 +#21 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dd35b5d8) at src/runtime/context.rs:180 +#22 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#23 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#24 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dd359e90, f=...) at src/runtime/context.rs:180 +#25 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#26 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dd35a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#27 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#28 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#29 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dd35a220) at src/runtime/blocking/task.rs:42 +#30 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c4a8) at src/runtime/task/core.rs:328 +#31 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c4a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#32 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c4a0, cx=...) at src/runtime/task/core.rs:317 +#33 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#34 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#35 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dd35a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#36 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#37 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#38 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#39 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c4a0, cx=...) at src/runtime/task/harness.rs:473 +#40 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dd35a5b0) at src/runtime/task/harness.rs:208 +#41 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#42 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#43 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#44 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#45 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#46 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de590, worker_thread_id=1) at src/runtime/blocking/pool.rs:513 +#47 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#48 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#49 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#50 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#51 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dd35aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#52 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#53 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#54 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#55 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#56 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#57 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#58 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#59 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#60 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#61 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1191 (Thread 0x7f91dd58f640 (LWP 2741193) "impwkr-v0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dd58b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dd58b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dd58b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140264673231248, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c190, mutex=0x7f91ea05c198, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c190, mutex_guard=0x7f91dd55a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c190, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c180) at src/runtime/scheduler/multi_thread/park.rs:148 +#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c180, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:119 +#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dd55a9a0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:67 +#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dd55ae98, core=0x7f956a841ff0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dd55ae98, core=0x7f956a841ff0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dd55ae98, core=0x7f956a841ff0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dd55c610, t=0x7f91dd55ae90, f=...) at src/runtime/context/scoped.rs:40 +#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dd55c5d8) at src/runtime/context.rs:180 +#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dd55ae90, f=...) at src/runtime/context.rs:180 +#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dd55b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dd55b220) at src/runtime/blocking/task.rs:42 +#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c428) at src/runtime/task/core.rs:328 +#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c428, f=...) at src/loom/std/unsafe_cell.rs:16 +#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c420, cx=...) at src/runtime/task/core.rs:317 +#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dd55b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c420, cx=...) at src/runtime/task/harness.rs:473 +#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dd55b5b0) at src/runtime/task/harness.rs:208 +#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de590, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 +#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dd55ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1190 (Thread 0x7f94c19ff640 (LWP 2741192) "rtkp"): +#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 +#1 0x00007f9583185778 in mio::sys::unix::selector::epoll::Selector::select (self=0x7f956919c238, events=0x7f956919c218, timeout=...) at src/sys/unix/selector/epoll.rs:97 +#2 0x00007f9583184fd1 in mio::poll::Poll::poll (self=0x7f956919c230, events=0x7f956919c218, timeout=...) at src/poll.rs:426 +#3 0x00007f9583167b7c in tokio::runtime::io::driver::Driver::turn (self=0x7f956919c218, handle=0x7f91ea093118, max_wait=...) at src/runtime/io/driver.rs:149 +#4 0x00007f95831677ee in tokio::runtime::io::driver::Driver::park (self=0x7f956919c218, rt_handle=0x7f91ea093118) at src/runtime/io/driver.rs:122 +#5 0x00007f958316a887 in tokio::runtime::signal::Driver::park (self=0x7f956919c218, handle=0x7f91ea093118) at src/runtime/signal/mod.rs:92 +#6 0x00007f9583152787 in tokio::runtime::process::Driver::park (self=0x7f956919c218, handle=0x7f91ea093118) at src/runtime/process.rs:32 +#7 0x00007f95830f1aca in tokio::runtime::driver::IoStack::park (self=0x7f956919c218, handle=0x7f91ea093118) at src/runtime/driver.rs:175 +#8 0x00007f958316a02e in tokio::runtime::time::Driver::park_internal (self=0x7f956919c218, rt_handle=0x7f91ea093118, limit=...) at src/runtime/time/mod.rs:235 +#9 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f956919c218, handle=0x7f91ea093118) at src/runtime/time/mod.rs:165 +#10 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f956919c210, handle=0x7f91ea093118) at src/runtime/driver.rs:332 +#11 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f956919c210, handle=0x7f91ea093118) at src/runtime/driver.rs:71 +#12 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f91ea05c060, driver=0x7f956919c210, handle=0x7f91ea093118) at src/runtime/scheduler/multi_thread/park.rs:184 +#13 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c060, handle=0x7f91ea093118) at src/runtime/scheduler/multi_thread/park.rs:117 +#14 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f94c19ca9a0, handle=0x7f91ea093118) at src/runtime/scheduler/multi_thread/park.rs:67 +#15 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f94c19cae98, core=0x7f956a841e10, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#16 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f94c19cae98, core=0x7f956a841e10) at src/runtime/scheduler/multi_thread/worker.rs:702 +#17 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f94c19cae98, core=0x7f956a841e10) at src/runtime/scheduler/multi_thread/worker.rs:553 +#18 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#19 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f94c19cc610, t=0x7f94c19cae90, f=...) at src/runtime/context/scoped.rs:40 +#20 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f94c19cc5d8) at src/runtime/context.rs:180 +#21 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#22 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#23 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f94c19cae90, f=...) at src/runtime/context.rs:180 +#24 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#25 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f94c19cb098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#26 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#27 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#28 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f94c19cb220) at src/runtime/blocking/task.rs:42 +#29 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c2a8) at src/runtime/task/core.rs:328 +#30 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c2a8, f=...) at src/loom/std/unsafe_cell.rs:16 +#31 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c2a0, cx=...) at src/runtime/task/core.rs:317 +#32 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#33 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#34 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f94c19cb398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#35 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#36 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#37 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#38 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c2a0, cx=...) at src/runtime/task/harness.rs:473 +#39 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f94c19cb5b0) at src/runtime/task/harness.rs:208 +#40 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#41 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#42 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#43 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#44 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#45 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de450, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 +#46 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#47 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#48 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#49 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#50 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f94c19cba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#51 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#52 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#53 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#54 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#55 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#56 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#57 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#58 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#59 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#60 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1189 (Thread 0x7f91dda10640 (LWP 2741191) "stats-monitor"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cfc2f4 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 0x00007f9583cea9cc in std::sys_common::thread_parking::futex::Parker::park_timeout () at library/std/src/sys_common/thread_parking/futex.rs:72 +#3 std::thread::park_timeout () at library/std/src/thread/mod.rs:1133 +#4 0x00007f957e13f3e0 in std::sync::mpmc::context::Context::wait_until (self=0x7f91dd9dae08, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:130 +#5 0x00007f957e42fc0a in std::sync::mpmc::list::{impl#3}::recv::{closure#1} (cx=0x7f91dd9dae08) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:444 +#6 0x00007f957e432120 in std::sync::mpmc::context::{impl#0}::with::{closure#0}, ()> (cx=, cx=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:50 +#7 std::sync::mpmc::context::{impl#0}::with::{closure#1}, ()> (cell=0x7f91dda0ca38) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:58 +#8 0x00007f957e43c74b in std::thread::local::LocalKey>>::try_with>, std::sync::mpmc::context::{impl#0}::with::{closure_env#1}, ()>, ()> (self=0x7f9583da0378, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957e43194e in std::sync::mpmc::context::Context::with, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:53 +#10 0x00007f957e42f6c9 in std::sync::mpmc::list::Channel::recv (self=0x7f92139a5a00, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:434 +#11 0x00007f957e13ea38 in std::sync::mpmc::Receiver::recv_deadline (self=0x7f91dd9db840, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/mod.rs:340 +#12 0x00007f957e13e7f1 in std::sync::mpmc::Receiver::recv_timeout (self=0x7f91dd9db840, timeout=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/mod.rs:323 +#13 0x00007f957dfe1a9a in std::sync::mpsc::Receiver::recv_timeout (self=0x7f91dd9db840, timeout=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpsc/mod.rs:909 +#14 0x00007f957b5ede37 in raftstore::store::worker::pd::{impl#12}::start::{closure#0}> () at components/raftstore/src/store/worker/pd.rs:713 +#15 0x00007f957df41a59 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#16 0x00007f957a80aef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#17 0x00007f957cb7a6d0 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#18 0x00007f957c9e4d54 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#19 0x00007f957be0a804 in std::panicking::try::do_call>, ()>, ()>>, ()> (data=0x7f91dd9dbed0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#20 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#21 0x00007f957bd876a8 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#22 0x00007f957cb77db0 in std::panic::catch_unwind>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#23 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#24 0x00007f957d67d7af in core::ops::function::FnOnce::call_once>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#25 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#26 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#28 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#29 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1188 (Thread 0x7f91ddc11640 (LWP 2741190) "apply-low-71"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91ddbd6288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91ddbd6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91ddbd6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91ddc0d798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91ddbd7868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91ddbd7868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91ddbd6ee0, batch=0x7f91ddbd6960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91ddbd6ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91ddbd9690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1187 (Thread 0x7f91de092640 (LWP 2741189) "apply-low-70"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91de057288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91de057288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91de057288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91de08e798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de058868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de058868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de057ee0, batch=0x7f91de057960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de057ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91de05a690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1186 (Thread 0x7f91de293640 (LWP 2741188) "apply-low-69"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91de258288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91de258288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91de258288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91de28f798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de259868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de259868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de258ee0, batch=0x7f91de258960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de258ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91de25b690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1185 (Thread 0x7f91de494640 (LWP 2741187) "apply-low-68"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91de459288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91de459288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91de459288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91de490798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de45a868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de45a868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de459ee0, batch=0x7f91de459960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de459ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91de45c690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1184 (Thread 0x7f91de695640 (LWP 2741186) "apply-low-67"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91de65a288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91de65a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91de65a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91de691798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de65b868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de65b868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de65aee0, batch=0x7f91de65a960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de65aee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91de65d690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1183 (Thread 0x7f91de896640 (LWP 2741185) "apply-low-66"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91de85b288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91de85b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91de85b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91de892798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de85c868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de85c868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de85bee0, batch=0x7f91de85b960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de85bee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91de85e690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1182 (Thread 0x7f91dea97640 (LWP 2741184) "apply-low-65"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dea5c288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dea5c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dea5c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dea93798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dea5d868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dea5d868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dea5cee0, batch=0x7f91dea5c960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dea5cee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dea5f690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1181 (Thread 0x7f91dec98640 (LWP 2741183) "apply-low-64"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dec5d288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dec5d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dec5d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dec94798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dec5e868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dec5e868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dec5dee0, batch=0x7f91dec5d960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dec5dee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dec60690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1180 (Thread 0x7f91dee99640 (LWP 2741182) "apply-low-63"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dee5e288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dee5e288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dee5e288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dee95798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dee5f868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dee5f868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dee5eee0, batch=0x7f91dee5e960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dee5eee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dee61690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1179 (Thread 0x7f91df09a640 (LWP 2741181) "apply-low-62"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91df05f288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91df05f288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91df05f288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91df096798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df060868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df060868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df05fee0, batch=0x7f91df05f960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df05fee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91df062690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1178 (Thread 0x7f91df29b640 (LWP 2741180) "apply-low-61"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91df260288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91df260288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91df260288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91df297798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df261868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df261868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df260ee0, batch=0x7f91df260960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df260ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91df263690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1177 (Thread 0x7f91df49c640 (LWP 2741179) "apply-low-60"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91df461288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91df461288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91df461288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91df498798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df462868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df462868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df461ee0, batch=0x7f91df461960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df461ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91df464690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1176 (Thread 0x7f91df91d640 (LWP 2741178) "apply-low-59"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91df8e2288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91df8e2288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91df8e2288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91df919798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df8e3868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df8e3868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df8e2ee0, batch=0x7f91df8e2960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df8e2ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91df8e5690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1175 (Thread 0x7f91dfb1e640 (LWP 2741177) "apply-low-58"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dfae3288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dfae3288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dfae3288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dfb1a798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfae4868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfae4868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfae3ee0, batch=0x7f91dfae3960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfae3ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dfae6690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1174 (Thread 0x7f91dfd1f640 (LWP 2741176) "apply-low-57"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dfce4288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dfce4288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dfce4288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dfd1b798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfce5868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfce5868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfce4ee0, batch=0x7f91dfce4960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfce4ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dfce7690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1173 (Thread 0x7f91dff20640 (LWP 2741175) "apply-low-56"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dfee5288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dfee5288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dfee5288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dff1c798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfee6868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfee6868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfee5ee0, batch=0x7f91dfee5960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfee5ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dfee8690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1172 (Thread 0x7f91e0121640 (LWP 2741174) "apply-low-55"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e00e6288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e00e6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e00e6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e011d798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e00e7868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e00e7868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e00e6ee0, batch=0x7f91e00e6960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e00e6ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e00e9690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1171 (Thread 0x7f91e0322640 (LWP 2741173) "apply-low-54"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e02e7288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e02e7288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e02e7288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e031e798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e02e8868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e02e8868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e02e7ee0, batch=0x7f91e02e7960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e02e7ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e02ea690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1170 (Thread 0x7f91e0523640 (LWP 2741172) "apply-low-53"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e04e8288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e04e8288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e04e8288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e051f798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e04e9868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e04e9868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e04e8ee0, batch=0x7f91e04e8960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e04e8ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e04eb690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1169 (Thread 0x7f91e0724640 (LWP 2741171) "apply-low-52"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e06e9288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e06e9288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e06e9288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e0720798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e06ea868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e06ea868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e06e9ee0, batch=0x7f91e06e9960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e06e9ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e06ec690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1168 (Thread 0x7f91e0925640 (LWP 2741170) "apply-low-51"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e08ea288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e08ea288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e08ea288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e0921798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e08eb868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e08eb868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e08eaee0, batch=0x7f91e08ea960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e08eaee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e08ed690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1167 (Thread 0x7f91e0b26640 (LWP 2741169) "apply-low-50"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e0aeb288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e0aeb288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e0aeb288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e0b22798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0aec868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0aec868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0aebee0, batch=0x7f91e0aeb960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0aebee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e0aee690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1166 (Thread 0x7f91e0d27640 (LWP 2741168) "apply-low-49"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e0cec288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e0cec288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e0cec288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e0d23798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0ced868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0ced868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0cecee0, batch=0x7f91e0cec960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0cecee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e0cef690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1165 (Thread 0x7f91e0f28640 (LWP 2741167) "apply-low-48"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e0eed288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e0eed288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e0eed288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e0f24798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0eee868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0eee868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0eedee0, batch=0x7f91e0eed960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0eedee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e0ef0690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1164 (Thread 0x7f91e1129640 (LWP 2741166) "apply-low-47"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e10ee288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e10ee288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e10ee288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1125798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e10ef868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e10ef868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e10eeee0, batch=0x7f91e10ee960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e10eeee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e10f1690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1163 (Thread 0x7f91e132a640 (LWP 2741165) "apply-low-46"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e12ef288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e12ef288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e12ef288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1326798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e12f0868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e12f0868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e12efee0, batch=0x7f91e12ef960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e12efee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e12f2690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1162 (Thread 0x7f91e152b640 (LWP 2741164) "apply-low-45"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e14f0288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e14f0288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e14f0288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1527798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e14f1868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e14f1868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e14f0ee0, batch=0x7f91e14f0960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e14f0ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e14f3690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1161 (Thread 0x7f91e172c640 (LWP 2741163) "apply-low-44"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e16f1288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e16f1288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e16f1288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1728798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e16f2868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e16f2868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e16f1ee0, batch=0x7f91e16f1960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e16f1ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e16f4690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1160 (Thread 0x7f91e192d640 (LWP 2741162) "apply-low-43"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e18f2288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e18f2288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e18f2288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1929798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e18f3868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e18f3868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e18f2ee0, batch=0x7f91e18f2960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e18f2ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e18f5690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1159 (Thread 0x7f91e1b2e640 (LWP 2741161) "apply-low-42"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e1af3288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e1af3288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e1af3288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1b2a798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1af4868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1af4868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1af3ee0, batch=0x7f91e1af3960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1af3ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e1af6690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1158 (Thread 0x7f91e1d2f640 (LWP 2741160) "apply-low-41"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e1cf4288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e1cf4288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e1cf4288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1d2b798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1cf5868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1cf5868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1cf4ee0, batch=0x7f91e1cf4960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1cf4ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e1cf7690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1157 (Thread 0x7f91e1f30640 (LWP 2741159) "apply-low-40"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e1ef5288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e1ef5288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e1ef5288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1f2c798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1ef6868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1ef6868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1ef5ee0, batch=0x7f91e1ef5960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1ef5ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e1ef8690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1156 (Thread 0x7f91e2131640 (LWP 2741158) "apply-low-39"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e20f6288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e20f6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e20f6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e212d798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e20f7868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e20f7868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e20f6ee0, batch=0x7f91e20f6960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e20f6ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e20f9690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1155 (Thread 0x7f91e2332640 (LWP 2741157) "apply-low-38"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e22f7288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e22f7288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e22f7288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e232e798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e22f8868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e22f8868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e22f7ee0, batch=0x7f91e22f7960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e22f7ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e22fa690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1154 (Thread 0x7f91e2533640 (LWP 2741156) "apply-low-37"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e24f8288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e24f8288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e24f8288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e252f798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e24f9868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e24f9868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e24f8ee0, batch=0x7f91e24f8960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e24f8ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e24fb690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1153 (Thread 0x7f91e2734640 (LWP 2741155) "apply-low-36"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e26f9288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e26f9288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e26f9288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e2730798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e26fa868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e26fa868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e26f9ee0, batch=0x7f91e26f9960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e26f9ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e26fc690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1152 (Thread 0x7f91e2935640 (LWP 2741154) "apply-low-35"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e28fa288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e28fa288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e28fa288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e2931798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e28fb868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e28fb868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e28faee0, batch=0x7f91e28fa960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e28faee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e28fd690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1151 (Thread 0x7f91e2b36640 (LWP 2741153) "apply-low-34"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e2afb288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e2afb288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e2afb288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e2b32798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2afc868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2afc868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2afbee0, batch=0x7f91e2afb960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2afbee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e2afe690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1150 (Thread 0x7f91e2d37640 (LWP 2741152) "apply-low-33"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e2cfc288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e2cfc288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e2cfc288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e2d33798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2cfd868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2cfd868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2cfcee0, batch=0x7f91e2cfc960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2cfcee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e2cff690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1149 (Thread 0x7f91e2f38640 (LWP 2741151) "apply-low-32"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e2efd288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e2efd288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e2efd288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e2f34798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2efe868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2efe868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2efdee0, batch=0x7f91e2efd960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2efdee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e2f00690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1148 (Thread 0x7f91e3139640 (LWP 2741150) "apply-low-31"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e30fe288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e30fe288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e30fe288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3135798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e30ff868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e30ff868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e30feee0, batch=0x7f91e30fe960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e30feee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3101690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1147 (Thread 0x7f91e333a640 (LWP 2741149) "apply-low-30"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e32ff288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e32ff288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e32ff288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3336798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3300868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3300868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e32ffee0, batch=0x7f91e32ff960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e32ffee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3302690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1146 (Thread 0x7f91e353b640 (LWP 2741148) "apply-low-29"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e3500288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e3500288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e3500288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3537798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3501868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3501868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3500ee0, batch=0x7f91e3500960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3500ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3503690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1145 (Thread 0x7f91e39bc640 (LWP 2741147) "apply-low-28"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e3981288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e3981288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e3981288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e39b8798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3982868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3982868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3981ee0, batch=0x7f91e3981960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3981ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3984690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1144 (Thread 0x7f91e3bbd640 (LWP 2741146) "apply-low-27"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e3b82288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e3b82288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e3b82288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3bb9798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3b83868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3b83868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3b82ee0, batch=0x7f91e3b82960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3b82ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3b85690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1143 (Thread 0x7f91e3dbe640 (LWP 2741145) "apply-low-26"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e3d83288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e3d83288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e3d83288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3dba798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3d84868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3d84868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3d83ee0, batch=0x7f91e3d83960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3d83ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3d86690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1142 (Thread 0x7f91e3fbf640 (LWP 2741144) "apply-low-25"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e3f84288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e3f84288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e3f84288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3fbb798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3f85868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3f85868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3f84ee0, batch=0x7f91e3f84960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3f84ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3f87690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1141 (Thread 0x7f91e41c0640 (LWP 2741143) "apply-low-24"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4185288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4185288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4185288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e41bc798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4186868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4186868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4185ee0, batch=0x7f91e4185960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4185ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e4188690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1140 (Thread 0x7f91e43c1640 (LWP 2741142) "apply-low-23"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4386288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4386288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4386288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e43bd798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4387868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4387868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4386ee0, batch=0x7f91e4386960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4386ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e4389690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1139 (Thread 0x7f91e45c2640 (LWP 2741141) "apply-low-22"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4587288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4587288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4587288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e45be798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4588868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4588868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4587ee0, batch=0x7f91e4587960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4587ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e458a690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1138 (Thread 0x7f91e47c3640 (LWP 2741140) "apply-low-21"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4788288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4788288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4788288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e47bf798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4789868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4789868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4788ee0, batch=0x7f91e4788960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4788ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e478b690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1137 (Thread 0x7f91e49c4640 (LWP 2741139) "apply-low-20"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4989288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4989288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4989288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e49c0798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e498a868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e498a868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4989ee0, batch=0x7f91e4989960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4989ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e498c690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1136 (Thread 0x7f91e4bc5640 (LWP 2741138) "apply-low-19"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4b8a288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4b8a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4b8a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e4bc1798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4b8b868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4b8b868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4b8aee0, batch=0x7f91e4b8a960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4b8aee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e4b8d690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1135 (Thread 0x7f91e4dc6640 (LWP 2741137) "apply-low-18"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4d8b288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4d8b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4d8b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e4dc2798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4d8c868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4d8c868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4d8bee0, batch=0x7f91e4d8b960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4d8bee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e4d8e690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1134 (Thread 0x7f91e4fc7640 (LWP 2741136) "apply-low-17"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4f8c288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4f8c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4f8c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e4fc3798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4f8d868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4f8d868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4f8cee0, batch=0x7f91e4f8c960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4f8cee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e4f8f690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1133 (Thread 0x7f91e5448640 (LWP 2741135) "apply-low-16"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e540d288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e540d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e540d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5444798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e540e868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e540e868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e540dee0, batch=0x7f91e540d960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e540dee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5410690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1132 (Thread 0x7f91e5649640 (LWP 2741134) "apply-low-15"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e560e288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e560e288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e560e288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5645798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e560f868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e560f868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e560eee0, batch=0x7f91e560e960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e560eee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5611690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1131 (Thread 0x7f91e584a640 (LWP 2741133) "apply-low-14"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e580f288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e580f288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e580f288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5846798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5810868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5810868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e580fee0, batch=0x7f91e580f960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e580fee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5812690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1130 (Thread 0x7f91e5a4b640 (LWP 2741132) "apply-low-13"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e5a10288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e5a10288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e5a10288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5a47798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5a11868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5a11868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5a10ee0, batch=0x7f91e5a10960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5a10ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5a13690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1129 (Thread 0x7f91e5c4c640 (LWP 2741131) "apply-low-12"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e5c11288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e5c11288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e5c11288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5c48798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5c12868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5c12868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5c11ee0, batch=0x7f91e5c11960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5c11ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5c14690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1128 (Thread 0x7f91e5e4d640 (LWP 2741130) "apply-low-11"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e5e12288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e5e12288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e5e12288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5e49798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5e13868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5e13868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5e12ee0, batch=0x7f91e5e12960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5e12ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5e15690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1127 (Thread 0x7f91e604e640 (LWP 2741129) "apply-low-10"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6013288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6013288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6013288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e604a798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6014868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6014868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6013ee0, batch=0x7f91e6013960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6013ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6016690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1126 (Thread 0x7f91e624f640 (LWP 2741128) "apply-low-9"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6214288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6214288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6214288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e624b798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6215868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6215868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6214ee0, batch=0x7f91e6214960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6214ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6217690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1125 (Thread 0x7f91e6450640 (LWP 2741127) "apply-low-8"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6415288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6415288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6415288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e644c798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6416868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6416868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6415ee0, batch=0x7f91e6415960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6415ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6418690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1124 (Thread 0x7f91e6651640 (LWP 2741126) "apply-low-7"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6616288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6616288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6616288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e664d798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6617868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6617868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6616ee0, batch=0x7f91e6616960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6616ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6619690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1123 (Thread 0x7f91e6852640 (LWP 2741125) "apply-low-6"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6817288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6817288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6817288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e684e798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6818868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6818868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6817ee0, batch=0x7f91e6817960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6817ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e681a690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1122 (Thread 0x7f91e6a53640 (LWP 2741124) "apply-low-5"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6a18288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6a18288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6a18288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e6a4f798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6a19868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6a19868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6a18ee0, batch=0x7f91e6a18960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6a18ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6a1b690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1121 (Thread 0x7f91e6c54640 (LWP 2741123) "apply-low-4"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6c19288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6c19288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6c19288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e6c50798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6c1a868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6c1a868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6c19ee0, batch=0x7f91e6c19960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6c19ee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6c1c690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1120 (Thread 0x7f91e6e55640 (LWP 2741122) "apply-low-3"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6e1a288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6e1a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6e1a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e6e51798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6e1b868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6e1b868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6e1aee0, batch=0x7f91e6e1a960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6e1aee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6e1d690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1119 (Thread 0x7f91e7056640 (LWP 2741121) "apply-low-2"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e701b288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e701b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e701b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e7052798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e701c868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e701c868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e701bee0, batch=0x7f91e701b960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e701bee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e701e690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1118 (Thread 0x7f91e7257640 (LWP 2741120) "apply-low-1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e721c288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e721c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e721c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e7253798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e721d868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e721d868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e721cee0, batch=0x7f91e721c960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e721cee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e721f690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1117 (Thread 0x7f91e7458640 (LWP 2741119) "apply-low-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e741d288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e741d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e741d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e7454798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e741e868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e741e868) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e741dee0, batch=0x7f91e741d960) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e741dee0) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e7420690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1116 (Thread 0x7f91e7659640 (LWP 2741118) "apply-3"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f957cb32c27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91e7655750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f957ca0db44 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91e7655750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f957a7d0e86 in parking_lot_core::parking_lot::park::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (thread_data=0x7f91e7655730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f957a7d0105 in parking_lot_core::parking_lot::with_thread_data, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (key=140279713980288, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f957b690e57 in tikv_util::mpsc::priority_queue::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e761f870) at components/tikv_util/src/mpsc/priority_queue.rs:175 +#7 0x00007f957cb363dd in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e761f868) at components/resource_control/src/channel.rs:158 +#8 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e761eee0, batch=0x7f91e761e960) at components/batch-system/src/batch.rs:373 +#9 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e761eee0) at components/batch-system/src/batch.rs:393 +#10 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#11 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#12 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#13 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#14 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#15 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e7621690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#16 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#17 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#18 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#19 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#20 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#21 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#22 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#23 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#24 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#25 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1115 (Thread 0x7f91e785a640 (LWP 2741117) "apply-2"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f957cb32c27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91e7856750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f957ca0db44 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91e7856750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f957a7d0e86 in parking_lot_core::parking_lot::park::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (thread_data=0x7f91e7856730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f957a7d0105 in parking_lot_core::parking_lot::with_thread_data, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (key=140279713980288, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f957b690e57 in tikv_util::mpsc::priority_queue::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7820870) at components/tikv_util/src/mpsc/priority_queue.rs:175 +#7 0x00007f957cb363dd in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7820868) at components/resource_control/src/channel.rs:158 +#8 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e781fee0, batch=0x7f91e781f960) at components/batch-system/src/batch.rs:373 +#9 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e781fee0) at components/batch-system/src/batch.rs:393 +#10 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#11 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#12 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#13 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#14 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#15 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e7822690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#16 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#17 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#18 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#19 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#20 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#21 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#22 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#23 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#24 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#25 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1114 (Thread 0x7f91e7a5b640 (LWP 2741116) "apply-1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f957cb32c27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91e7a57750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f957ca0db44 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91e7a57750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f957a7d0e86 in parking_lot_core::parking_lot::park::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (thread_data=0x7f91e7a57730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f957a7d0105 in parking_lot_core::parking_lot::with_thread_data, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (key=140279713980288, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f957b690e57 in tikv_util::mpsc::priority_queue::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7a21870) at components/tikv_util/src/mpsc/priority_queue.rs:175 +#7 0x00007f957cb363dd in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7a21868) at components/resource_control/src/channel.rs:158 +#8 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e7a20ee0, batch=0x7f91e7a20960) at components/batch-system/src/batch.rs:373 +#9 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e7a20ee0) at components/batch-system/src/batch.rs:393 +#10 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#11 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#12 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#13 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#14 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#15 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e7a23690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#16 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#17 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#18 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#19 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#20 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#21 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#22 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#23 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#24 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#25 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1113 (Thread 0x7f91e7c5c640 (LWP 2741115) "apply-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f957cb32c27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91e7c58750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f957ca0db44 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91e7c58750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f957a7d0e86 in parking_lot_core::parking_lot::park::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (thread_data=0x7f91e7c58730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f957a7d0105 in parking_lot_core::parking_lot::with_thread_data, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (key=140279713980288, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f957b690e57 in tikv_util::mpsc::priority_queue::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7c22870) at components/tikv_util/src/mpsc/priority_queue.rs:175 +#7 0x00007f957cb363dd in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7c22868) at components/resource_control/src/channel.rs:158 +#8 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e7c21ee0, batch=0x7f91e7c21960) at components/batch-system/src/batch.rs:373 +#9 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e7c21ee0) at components/batch-system/src/batch.rs:393 +#10 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 +#11 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#12 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#13 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#14 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#15 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e7c24690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#16 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#17 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#18 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#19 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#20 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#21 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#22 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#23 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#24 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#25 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1112 (Thread 0x7f91e7e5d640 (LWP 2741114) "time updater"): +#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 +#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 +#2 0x00007f9583cea820 in std::sys::unix::thread::Thread::sleep () at library/std/src/sys/unix/thread.rs:243 +#3 std::thread::sleep () at library/std/src/thread/mod.rs:870 +#4 0x00007f9582bdbe50 in prometheus::timer::ensure_updater::{closure#0} () at src/timer.rs:57 +#5 0x00007f9582bf32e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#6 0x00007f9582bf5a27 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#7 0x00007f9582c14c67 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#8 0x00007f9582bb5d0d in std::panicking::try::do_call>, ()> (data=0x7f91e7e29af0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#9 0x00007f9582be891b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#10 0x00007f9582bb5ca6 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#11 0x00007f9582bf5879 in std::panic::catch_unwind>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#12 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#13 0x00007f9582bb62af in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#14 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#15 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#16 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#17 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#18 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1111 (Thread 0x7f91e805e640 (LWP 2741113) "raftstore-135-3"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e8006378, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d658057 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::store::StoreFsm>> (cx=0x7f91e8006378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d645832 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::store::StoreFsm>>, ()> (cx=0x7f91e8006378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d647163 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::store::StoreFsm>>, ()> (cell=0x7f91e805a798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86375b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::store::StoreFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d641aee in crossbeam_channel::context::Context::with, raftstore::store::fsm::store::StoreFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d657b8f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f956a8cfe00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3cfde in crossbeam_channel::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e800a378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb36512 in resource_control::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e800a378) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a96d3 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::fetch_fsm, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8006fd8, batch=0x7f91e8006a50) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a65d9 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::poll, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8006fd8) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99a9b0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df423f6 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae87 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a66d in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e3441 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957bdf0311 in std::panicking::try::do_call, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (data=0x7f91e8013ff0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bd874f2 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb781ce in std::panic::catch_unwind, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e3cf in core::ops::function::FnOnce::call_once, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1110 (Thread 0x7f91e825f640 (LWP 2741112) "raftstore-135-2"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e8207378, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d658057 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::store::StoreFsm>> (cx=0x7f91e8207378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d645832 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::store::StoreFsm>>, ()> (cx=0x7f91e8207378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d647163 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::store::StoreFsm>>, ()> (cell=0x7f91e825b798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86375b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::store::StoreFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d641aee in crossbeam_channel::context::Context::with, raftstore::store::fsm::store::StoreFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d657b8f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f956a8cfe00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3cfde in crossbeam_channel::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e820b378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb36512 in resource_control::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e820b378) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a96d3 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::fetch_fsm, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8207fd8, batch=0x7f91e8207a50) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a65d9 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::poll, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8207fd8) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99a9b0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df423f6 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae87 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a66d in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e3441 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957bdf0311 in std::panicking::try::do_call, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (data=0x7f91e8214ff0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bd874f2 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb781ce in std::panic::catch_unwind, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e3cf in core::ops::function::FnOnce::call_once, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1109 (Thread 0x7f91e8460640 (LWP 2741111) "raftstore-135-1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e8408378, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d658057 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::store::StoreFsm>> (cx=0x7f91e8408378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d645832 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::store::StoreFsm>>, ()> (cx=0x7f91e8408378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d647163 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::store::StoreFsm>>, ()> (cell=0x7f91e845c798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86375b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::store::StoreFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d641aee in crossbeam_channel::context::Context::with, raftstore::store::fsm::store::StoreFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d657b8f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f956a8cfe00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3cfde in crossbeam_channel::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e840c378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb36512 in resource_control::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e840c378) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a96d3 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::fetch_fsm, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8408fd8, batch=0x7f91e8408a50) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a65d9 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::poll, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8408fd8) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99a9b0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df423f6 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae87 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a66d in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e3441 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957bdf0311 in std::panicking::try::do_call, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (data=0x7f91e8415ff0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bd874f2 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb781ce in std::panic::catch_unwind, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e3cf in core::ops::function::FnOnce::call_once, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1108 (Thread 0x7f91e8661640 (LWP 2741110) "raftstore-135-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e8609378, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f957d658057 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::store::StoreFsm>> (cx=0x7f91e8609378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 +#6 0x00007f957d645832 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::store::StoreFsm>>, ()> (cx=0x7f91e8609378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f957d647163 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::store::StoreFsm>>, ()> (cell=0x7f91e865d798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f957a86375b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::store::StoreFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957d641aee in crossbeam_channel::context::Context::with, raftstore::store::fsm::store::StoreFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f957d657b8f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f956a8cfe00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 +#11 0x00007f957cb3cfde in crossbeam_channel::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e860d378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 +#12 0x00007f957cb36512 in resource_control::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e860d378) at components/resource_control/src/channel.rs:157 +#13 0x00007f957b9a96d3 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::fetch_fsm, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8609fd8, batch=0x7f91e8609a50) at components/batch-system/src/batch.rs:373 +#14 0x00007f957b9a65d9 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::poll, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8609fd8) at components/batch-system/src/batch.rs:393 +#15 0x00007f957b99a9b0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> () at components/batch-system/src/batch.rs:588 +#16 0x00007f957df423f6 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#17 0x00007f957a80ae87 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#18 0x00007f957cb7a66d in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#19 0x00007f957c9e3441 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#20 0x00007f957bdf0311 in std::panicking::try::do_call, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (data=0x7f91e8616ff0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#22 0x00007f957bd874f2 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#23 0x00007f957cb781ce in std::panic::catch_unwind, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#25 0x00007f957d67e3cf in core::ops::function::FnOnce::call_once, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1107 (Thread 0x7f91e8862640 (LWP 2741109) "steady-timer"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cfc2f4 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 0x00007f9583d0039a in std::sys::unix::locks::futex_condvar::Condvar::wait_optional_timeout () at library/std/src/sys/unix/locks/futex_condvar.rs:49 +#3 std::sys::unix::locks::futex_condvar::Condvar::wait_timeout () at library/std/src/sys/unix/locks/futex_condvar.rs:37 +#4 0x00007f958327577d in std::sync::condvar::Condvar::wait_timeout<()> (self=0x7f94aeac63e0, guard=..., dur=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/condvar.rs:385 +#5 0x00007f958327db79 in crossbeam_utils::sync::parker::Inner::park (self=0x7f94aeac63d0, deadline=...) at src/sync/parker.rs:367 +#6 0x00007f958327d5b2 in crossbeam_utils::sync::parker::Parker::park_deadline (self=0x7f91e885e5c0, deadline=...) at src/sync/parker.rs:145 +#7 0x00007f958327d567 in crossbeam_utils::sync::parker::Parker::park_timeout (self=0x7f91e885e5c0, timeout=...) at src/sync/parker.rs:125 +#8 0x00007f9582a0a02a in tokio_executor::park::{impl#3}::park_timeout::{closure#0} (inner=0x7f91e885e5c0) at src/park.rs:215 +#9 0x00007f9582a09eeb in tokio_executor::park::{impl#2}::with_current::{closure#0} (inner=0x7f91e885e5c0) at src/park.rs:196 +#10 0x00007f9582a0a347 in std::thread::local::LocalKey::try_with, ()> (self=0x7f958406cfc8, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#11 0x00007f9582a0a214 in std::thread::local::LocalKey::with, ()> (self=0x7f958406cfc8, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#12 0x00007f9582a09ec3 in tokio_executor::park::ParkThread::with_current (self=0x7f91e882e810, f=...) at src/park.rs:196 +#13 0x00007f9582a09fe3 in tokio_executor::park::{impl#3}::park_timeout (self=0x7f91e882e810, duration=...) at src/park.rs:215 +#14 0x00007f95828c9a5c in tokio_timer::timer::{impl#4}::park (self=0x7f91e882e7d8) at /home/xzx/.cargo/git/checkouts/tokio-8e927faba632ed16/4394380/tokio-timer/src/timer/mod.rs:369 +#15 0x00007f95828a52c9 in tokio_timer::timer::Timer::turn (self=0x7f91e882e7d8, max_wait=...) at /home/xzx/.cargo/git/checkouts/tokio-8e927faba632ed16/4394380/tokio-timer/src/timer/mod.rs:256 +#16 0x00007f958289e7f9 in tikv_util::timer::start_timer_thread::{closure#0} () at components/tikv_util/src/timer.rs:259 +#17 0x00007f95828de219 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, ()> () at components/tikv_util/src/sys/thread.rs:441 +#18 0x00007f95828ad357 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#19 0x00007f95828b81ad in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#20 0x00007f958285eec1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#21 0x00007f958286a625 in std::panicking::try::do_call, ()>, ()>>, ()> (data=0x7f91e882e9e0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#22 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#23 0x00007f9582869e7f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#24 0x00007f95828b72e1 in std::panic::catch_unwind, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#25 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#26 0x00007f95827ed88f in core::ops::function::FnOnce::call_once, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#27 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#28 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#29 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#30 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#31 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1106 (Thread 0x7f91e8a63640 (LWP 2741052) "store-writer-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f957cb32c27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91e8a5f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f957ca0db44 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91e8a5f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f957a7d2746 in parking_lot_core::parking_lot::park::{closure#0}>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}>> (thread_data=0x7f91e8a5f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f957a7cff75 in parking_lot_core::parking_lot::with_thread_data>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}>> (key=140266244010880, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f957b690f6a in tikv_util::mpsc::priority_queue::Receiver>::recv> (self=0x7f91e8a27660) at components/tikv_util/src/mpsc/priority_queue.rs:175 +#7 0x00007f957cb3644d in resource_control::channel::Receiver>::recv> (self=0x7f91e8a27658) at components/resource_control/src/channel.rs:158 +#8 0x00007f957b682b5f in raftstore::store::async_io::write::Worker, tikv::server::transport::ServerTransport, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>::run, tikv::server::transport::ServerTransport, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>> (self=0x7f91e8a26ee8) at components/raftstore/src/store/async_io/write.rs:762 +#9 0x00007f957b671ade in raftstore::store::async_io::write::{impl#10}::increase_to::{closure#0}::{closure#0}, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter> () at components/raftstore/src/store/async_io/write.rs:1193 +#10 0x00007f957df42539 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#11 0x00007f957a80adf7 in std::sys_common::backtrace::__rust_begin_short_backtrace, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#12 0x00007f957cb7a6a0 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#13 0x00007f957c9da2a4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#14 0x00007f957be16364 in std::panicking::try::do_call, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()>>, ()> (data=0x7f91e8a2a240) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#15 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#16 0x00007f957bdb9517 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#17 0x00007f957cb779a9 in std::panic::catch_unwind, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#18 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#19 0x00007f957d67f34f in core::ops::function::FnOnce::call_once, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#20 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#22 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#23 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#24 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1105 (Thread 0x7f91eec64640 (LWP 2741048) "refreash-config"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91eec60750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91eec60750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91eec60730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248948432, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91eec2fc70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91eec2fb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91eec30170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1104 (Thread 0x7f91eee65640 (LWP 2741047) "raftlog-fetch-w"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91eee61750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91eee61750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91eee61730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248947792, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91eee30c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91eee30b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91eee31170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1103 (Thread 0x7f91ef066640 (LWP 2741046) "region-worker-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ef062750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ef062750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ef062730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248947152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ef031c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91ef031b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91ef032170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1102 (Thread 0x7f91ef4e7640 (LWP 2741045) "cleanup-worker-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ef4e3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ef4e3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ef4e3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248946512, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ef4b2c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91ef4b2b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91ef4b3170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1101 (Thread 0x7f91ef6e8640 (LWP 2741044) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ef6e4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ef6e4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ef6e4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ef6b3c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91ef6b3b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91ef6b4170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1100 (Thread 0x7f91ef8e9640 (LWP 2741043) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ef8e5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ef8e5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ef8e5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ef8b4c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91ef8b4b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91ef8b5170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1099 (Thread 0x7f91efaea640 (LWP 2741042) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91efae6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91efae6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91efae6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91efab5c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91efab5b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91efab6170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1098 (Thread 0x7f91efceb640 (LWP 2741041) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91efce7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91efce7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91efce7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91efcb6c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91efcb6b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91efcb7170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1097 (Thread 0x7f91efeec640 (LWP 2741040) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91efee8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91efee8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91efee8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91efeb7c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91efeb7b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91efeb8170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1096 (Thread 0x7f91f00ed640 (LWP 2741039) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f00e9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f00e9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f00e9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f00b8c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f00b8b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f00b9170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1095 (Thread 0x7f91f02ee640 (LWP 2741038) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f02ea750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f02ea750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f02ea730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f02b9c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f02b9b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f02ba170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1094 (Thread 0x7f91f04ef640 (LWP 2741037) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f04eb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f04eb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f04eb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f04bac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f04bab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f04bb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1093 (Thread 0x7f91f06f0640 (LWP 2741036) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f06ec750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f06ec750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f06ec730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f06bbc70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f06bbb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f06bc170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1092 (Thread 0x7f91f08f1640 (LWP 2741035) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f08ed750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f08ed750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f08ed730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f08bcc70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f08bcb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f08bd170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1091 (Thread 0x7f91f0af2640 (LWP 2741034) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f0aee750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f0aee750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f0aee730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f0abdc70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f0abdb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f0abe170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1090 (Thread 0x7f91f0cf3640 (LWP 2741033) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f0cef750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f0cef750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f0cef730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f0cbec70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f0cbeb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f0cbf170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1089 (Thread 0x7f91f0ef4640 (LWP 2741032) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f0ef0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f0ef0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f0ef0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f0ebfc70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f0ebfb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f0ec0170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1088 (Thread 0x7f91f10f5640 (LWP 2741031) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f10f1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f10f1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f10f1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f10c0c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f10c0b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f10c1170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1087 (Thread 0x7f91f12f6640 (LWP 2741030) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f12f2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f12f2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f12f2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f12c1c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f12c1b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f12c2170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1086 (Thread 0x7f91f14f7640 (LWP 2741029) "snap-generator-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f14f3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f14f3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f14f3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f14c2c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f14c2b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f14c3170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1085 (Thread 0x7f91f16f8640 (LWP 2741028) "purge-worker-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f16f4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f16f4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f16f4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945232, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f16c3c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f16c3b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f16c4170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1084 (Thread 0x7f91f18f9640 (LWP 2741027) "region-task-15"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f18f5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f18f5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f18f5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f18c5648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f18c5648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f18c5860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1083 (Thread 0x7f91f1afa640 (LWP 2741026) "region-task-14"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f1af6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f1af6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f1af6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f1ac6648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f1ac6648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f1ac6860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1082 (Thread 0x7f91f1cfb640 (LWP 2741025) "region-task-13"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f1cf7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f1cf7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f1cf7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f1cc7648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f1cc7648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f1cc7860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1081 (Thread 0x7f91f1efc640 (LWP 2741024) "region-task-12"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f1ef8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f1ef8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f1ef8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f1ec8648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f1ec8648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f1ec8860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1080 (Thread 0x7f91f20fd640 (LWP 2741023) "region-task-11"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f20f9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f20f9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f20f9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f20c9648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f20c9648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f20c9860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1079 (Thread 0x7f91f22fe640 (LWP 2741022) "region-task-10"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f22fa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f22fa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f22fa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f22ca648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f22ca648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f22ca860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1078 (Thread 0x7f91f24ff640 (LWP 2741021) "region-task-9"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f24fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f24fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f24fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f24cb648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f24cb648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f24cb860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1077 (Thread 0x7f91f2700640 (LWP 2741020) "region-task-8"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f26fc750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f26fc750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f26fc730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f26cc648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f26cc648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f26cc860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1076 (Thread 0x7f91f2901640 (LWP 2741019) "region-task-7"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f28fd750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f28fd750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f28fd730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f28cd648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f28cd648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f28cd860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1075 (Thread 0x7f91f2b02640 (LWP 2741018) "region-task-6"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f2afe750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f2afe750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f2afe730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f2ace648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f2ace648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f2ace860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1074 (Thread 0x7f91f2d03640 (LWP 2741017) "region-task-5"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f2cff750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f2cff750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f2cff730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f2ccf648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f2ccf648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f2ccf860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1073 (Thread 0x7f91f2f04640 (LWP 2741016) "region-task-4"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f2f00750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f2f00750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f2f00730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f2ed0648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f2ed0648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f2ed0860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1072 (Thread 0x7f91f3105640 (LWP 2741015) "region-task-3"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3101750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3101750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3101730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f30d1648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f30d1648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f30d1860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1071 (Thread 0x7f91f3306640 (LWP 2741014) "region-task-2"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3302750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3302750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3302730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f32d2648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f32d2648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f32d2860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1070 (Thread 0x7f91f3507640 (LWP 2741013) "region-task-1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3503750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3503750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3503730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f34d3648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f34d3648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f34d3860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1069 (Thread 0x7f91f3708640 (LWP 2741012) "region-task-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3704750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3704750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3704730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f36d4648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f36d4648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f36d4860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1068 (Thread 0x7f91f3909640 (LWP 2741011) "raft-stream-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3905750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3905750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3905730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944432, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f38d5648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f38d5648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f38d5860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1067 (Thread 0x7f91f3b0a640 (LWP 2741004) "snap-handler-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3b06750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3b06750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3b06730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248943792, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f3ad5c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f3ad5b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f3ad6170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1066 (Thread 0x7f91f3d0b640 (LWP 2741003) "transport-stats"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3d07750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3d07750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91f3d07730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140279560138632, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f956159ef88, mutex=0x7f956159ef90, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f956159ef88, mutex_guard=0x7f91f3cd6368) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f956159ef88, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830ef2f6 in tokio::runtime::park::Inner::park (self=0x7f956159ef80) at src/runtime/park.rs:116 +#10 0x00007f95830ef113 in tokio::runtime::park::ParkThread::park (self=0x7f95615941a0) at src/runtime/park.rs:62 +#11 0x00007f95830f1adf in tokio::runtime::driver::IoStack::park (self=0x7f9561594198, handle=0x7f956a8ced18) at src/runtime/driver.rs:176 +#12 0x00007f95830f2090 in tokio::runtime::driver::TimeDriver::park (self=0x7f9561594190, handle=0x7f956a8ced18) at src/runtime/driver.rs:333 +#13 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f9561594190, handle=0x7f956a8ced18) at src/runtime/driver.rs:71 +#14 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f9247f23010, driver=0x7f9561594190, handle=0x7f956a8ced18) at src/runtime/scheduler/multi_thread/park.rs:184 +#15 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f9247f23010, handle=0x7f956a8ced18) at src/runtime/scheduler/multi_thread/park.rs:117 +#16 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91f3cd69a0, handle=0x7f956a8ced18) at src/runtime/scheduler/multi_thread/park.rs:67 +#17 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91f3cd6e98, core=0x7f956159b920, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#18 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91f3cd6e98, core=0x7f956159b920) at src/runtime/scheduler/multi_thread/worker.rs:702 +#19 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91f3cd6e98, core=0x7f956159b920) at src/runtime/scheduler/multi_thread/worker.rs:553 +#20 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#21 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91f3cd8610, t=0x7f91f3cd6e90, f=...) at src/runtime/context/scoped.rs:40 +#22 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91f3cd85d8) at src/runtime/context.rs:180 +#23 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#24 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#25 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91f3cd6e90, f=...) at src/runtime/context.rs:180 +#26 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#27 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91f3cd7098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#28 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#29 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#30 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91f3cd7220) at src/runtime/blocking/task.rs:42 +#31 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594228) at src/runtime/task/core.rs:328 +#32 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594228, f=...) at src/loom/std/unsafe_cell.rs:16 +#33 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594220, cx=...) at src/runtime/task/core.rs:317 +#34 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#35 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#36 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91f3cd7398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#37 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#38 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#39 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#40 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594220, cx=...) at src/runtime/task/harness.rs:473 +#41 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91f3cd75b0) at src/runtime/task/harness.rs:208 +#42 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#43 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#44 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#45 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#46 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#47 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f95615afc50, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 +#48 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#49 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#50 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#51 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#52 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91f3cd7a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#53 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#54 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#55 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#56 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#57 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#58 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#59 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#60 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#61 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#62 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1065 (Thread 0x7f91f3f0c640 (LWP 2741001) "sst_import_misc"): +#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 +#1 0x00007f9583185778 in mio::sys::unix::selector::epoll::Selector::select (self=0x7f9561507f38, events=0x7f9561507f18, timeout=...) at src/sys/unix/selector/epoll.rs:97 +#2 0x00007f9583184fd1 in mio::poll::Poll::poll (self=0x7f9561507f30, events=0x7f9561507f18, timeout=...) at src/poll.rs:426 +#3 0x00007f9583167b7c in tokio::runtime::io::driver::Driver::turn (self=0x7f9561507f18, handle=0x7f956a8ceb18, max_wait=...) at src/runtime/io/driver.rs:149 +#4 0x00007f958316786a in tokio::runtime::io::driver::Driver::park_timeout (self=0x7f9561507f18, rt_handle=0x7f956a8ceb18, duration=...) at src/runtime/io/driver.rs:127 +#5 0x00007f958316a8c0 in tokio::runtime::signal::Driver::park_timeout (self=0x7f9561507f18, handle=0x7f956a8ceb18, duration=...) at src/runtime/signal/mod.rs:97 +#6 0x00007f95831527c0 in tokio::runtime::process::Driver::park_timeout (self=0x7f9561507f18, handle=0x7f956a8ceb18, duration=...) at src/runtime/process.rs:37 +#7 0x00007f95830f1b55 in tokio::runtime::driver::IoStack::park_timeout (self=0x7f9561507f18, handle=0x7f956a8ceb18, duration=...) at src/runtime/driver.rs:182 +#8 0x00007f958316a49c in tokio::runtime::time::Driver::park_thread_timeout (self=0x7f9561507f18, rt_handle=0x7f956a8ceb18, duration=...) at src/runtime/time/mod.rs:270 +#9 0x00007f958316a1f5 in tokio::runtime::time::Driver::park_internal (self=0x7f9561507f18, rt_handle=0x7f956a8ceb18, limit=...) at src/runtime/time/mod.rs:226 +#10 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f9561507f18, handle=0x7f956a8ceb18) at src/runtime/time/mod.rs:165 +#11 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f9561507f10, handle=0x7f956a8ceb18) at src/runtime/driver.rs:332 +#12 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f9561507f10, handle=0x7f956a8ceb18) at src/runtime/driver.rs:71 +#13 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f956159ead0, driver=0x7f9561507f10, handle=0x7f956a8ceb18) at src/runtime/scheduler/multi_thread/park.rs:184 +#14 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f956159ead0, handle=0x7f956a8ceb18) at src/runtime/scheduler/multi_thread/park.rs:117 +#15 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91f3ed79a0, handle=0x7f956a8ceb18) at src/runtime/scheduler/multi_thread/park.rs:67 +#16 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91f3ed7e98, core=0x7f956159b6a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#17 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91f3ed7e98, core=0x7f956159b6a0) at src/runtime/scheduler/multi_thread/worker.rs:702 +#18 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91f3ed7e98, core=0x7f956159b6a0) at src/runtime/scheduler/multi_thread/worker.rs:553 +#19 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#20 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91f3ed9610, t=0x7f91f3ed7e90, f=...) at src/runtime/context/scoped.rs:40 +#21 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91f3ed95d8) at src/runtime/context.rs:180 +#22 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#23 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#24 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91f3ed7e90, f=...) at src/runtime/context.rs:180 +#25 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#26 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91f3ed8098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#27 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#28 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#29 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91f3ed8220) at src/runtime/blocking/task.rs:42 +#30 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507fa8) at src/runtime/task/core.rs:328 +#31 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507fa8, f=...) at src/loom/std/unsafe_cell.rs:16 +#32 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507fa0, cx=...) at src/runtime/task/core.rs:317 +#33 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#34 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#35 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91f3ed8398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#36 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#37 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#38 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#39 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507fa0, cx=...) at src/runtime/task/harness.rs:473 +#40 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91f3ed85b0) at src/runtime/task/harness.rs:208 +#41 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#42 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#43 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#44 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#45 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#46 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f95615af9d0, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 +#47 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#48 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#49 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#50 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#51 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91f3ed8a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#52 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#53 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#54 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#55 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#56 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#57 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#58 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#59 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#60 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#61 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1064 (Thread 0x7f91f410d640 (LWP 2740998) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f4109750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f4109750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f4109730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f40d7be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f40d7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f40d85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1063 (Thread 0x7f91f430e640 (LWP 2740997) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f430a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f430a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f430a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f42d8be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f42d8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f42d95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1062 (Thread 0x7f91f450f640 (LWP 2740996) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f450b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f450b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f450b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f44d9be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f44d9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f44da5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1061 (Thread 0x7f91f4710640 (LWP 2740995) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f470c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f470c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f470c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f46dabe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f46da938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f46db5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1060 (Thread 0x7f91f4911640 (LWP 2740994) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f490d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f490d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f490d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f48dbbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f48db938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f48dc5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1059 (Thread 0x7f91f4b12640 (LWP 2740993) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f4b0e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f4b0e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f4b0e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f4adcbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f4adc938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f4add5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1058 (Thread 0x7f91f4d13640 (LWP 2740992) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f4d0f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f4d0f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f4d0f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f4cddbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f4cdd938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f4cde5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1057 (Thread 0x7f91f4f14640 (LWP 2740991) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f4f10750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f4f10750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f4f10730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f4edebe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f4ede938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f4edf5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1056 (Thread 0x7f91f5115640 (LWP 2740990) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5111750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5111750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5111730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f50dfbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f50df938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f50e05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1055 (Thread 0x7f91f5316640 (LWP 2740989) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5312750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5312750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5312730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f52e0be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f52e0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f52e15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1054 (Thread 0x7f91f5517640 (LWP 2740988) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5513750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5513750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5513730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f54e1be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f54e1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f54e25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1053 (Thread 0x7f91f5718640 (LWP 2740987) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5714750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5714750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5714730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f56e2be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f56e2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f56e35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1052 (Thread 0x7f91f5919640 (LWP 2740986) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5915750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5915750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5915730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f58e3be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f58e3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f58e45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1051 (Thread 0x7f91f5b1a640 (LWP 2740985) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5b16750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5b16750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5b16730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f5ae4be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f5ae4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f5ae55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1050 (Thread 0x7f91f5d1b640 (LWP 2740984) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5d17750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5d17750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5d17730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f5ce5be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f5ce5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f5ce65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1049 (Thread 0x7f91f5f1c640 (LWP 2740983) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5f18750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5f18750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5f18730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f5ee6be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f5ee6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f5ee75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1048 (Thread 0x7f91f611d640 (LWP 2740982) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f6119750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f6119750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f6119730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f60e7be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f60e7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f60e85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1047 (Thread 0x7f91f631e640 (LWP 2740981) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f631a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f631a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f631a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f62e8be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f62e8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f62e95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1046 (Thread 0x7f91f651f640 (LWP 2740980) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f651b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f651b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f651b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f64e9be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f64e9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f64ea5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1045 (Thread 0x7f91f6720640 (LWP 2740979) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f671c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f671c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f671c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f66eabe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f66ea938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f66eb5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1044 (Thread 0x7f91f6921640 (LWP 2740978) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f691d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f691d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f691d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f68ebbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f68eb938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f68ec5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1043 (Thread 0x7f91f6b22640 (LWP 2740977) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f6b1e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f6b1e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f6b1e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f6aecbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f6aec938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f6aed5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1042 (Thread 0x7f91f6d23640 (LWP 2740976) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f6d1f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f6d1f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f6d1f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f6cedbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f6ced938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f6cee5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1041 (Thread 0x7f91f6f24640 (LWP 2740975) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f6f20750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f6f20750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f6f20730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f6eeebe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f6eee938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f6eef5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1040 (Thread 0x7f91f7125640 (LWP 2740974) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7121750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7121750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7121730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f70efbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f70ef938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f70f05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1039 (Thread 0x7f91f7326640 (LWP 2740973) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7322750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7322750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7322730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f72f0be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f72f0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f72f15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1038 (Thread 0x7f91f7527640 (LWP 2740972) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7523750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7523750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7523730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f74f1be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f74f1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f74f25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1037 (Thread 0x7f91f7728640 (LWP 2740971) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7724750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7724750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7724730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f76f2be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f76f2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f76f35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1036 (Thread 0x7f91f7929640 (LWP 2740970) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7925750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7925750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7925730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f78f3be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f78f3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f78f45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1035 (Thread 0x7f91f7b2a640 (LWP 2740969) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7b26750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7b26750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7b26730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f7af4be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f7af4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f7af55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1034 (Thread 0x7f91f7d2b640 (LWP 2740968) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7d27750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7d27750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7d27730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f7cf5be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f7cf5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f7cf65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1033 (Thread 0x7f91f7f2c640 (LWP 2740967) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7f28750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7f28750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7f28730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f7ef6be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f7ef6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f7ef75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1032 (Thread 0x7f91f812d640 (LWP 2740966) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f8129750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f8129750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f8129730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f80f7be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f80f7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f80f85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1031 (Thread 0x7f91f832e640 (LWP 2740965) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f832a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f832a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f832a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f82f8be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f82f8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f82f95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1030 (Thread 0x7f91f852f640 (LWP 2740964) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f852b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f852b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f852b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f84f9be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f84f9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f84fa5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1029 (Thread 0x7f91f8730640 (LWP 2740963) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f872c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f872c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f872c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f86fabe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f86fa938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f86fb5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1028 (Thread 0x7f91f8931640 (LWP 2740962) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f892d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f892d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f892d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f88fbbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f88fb938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f88fc5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1027 (Thread 0x7f91f8b32640 (LWP 2740961) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f8b2e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f8b2e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f8b2e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f8afcbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f8afc938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f8afd5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1026 (Thread 0x7f91f8d33640 (LWP 2740960) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f8d2f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f8d2f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f8d2f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f8cfdbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f8cfd938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f8cfe5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1025 (Thread 0x7f91f8f34640 (LWP 2740959) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f8f30750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f8f30750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f8f30730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f8efebe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f8efe938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f8eff5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1024 (Thread 0x7f91f9135640 (LWP 2740958) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9131750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9131750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9131730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f90ffbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f90ff938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f91005d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1023 (Thread 0x7f91f9336640 (LWP 2740957) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9332750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9332750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9332730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9300be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9300938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f93015d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1022 (Thread 0x7f91f9537640 (LWP 2740956) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9533750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9533750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9533730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9501be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9501938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f95025d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1021 (Thread 0x7f91f9738640 (LWP 2740955) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9734750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9734750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9734730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9702be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9702938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f97035d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1020 (Thread 0x7f91f9939640 (LWP 2740954) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9935750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9935750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9935730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9903be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9903938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f99045d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1019 (Thread 0x7f91f9b3a640 (LWP 2740953) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9b36750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9b36750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9b36730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9b04be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9b04938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f9b055d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1018 (Thread 0x7f91f9d3b640 (LWP 2740952) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9d37750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9d37750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9d37730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9d05be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9d05938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f9d065d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1017 (Thread 0x7f91f9f3c640 (LWP 2740951) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9f38750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9f38750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9f38730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9f06be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9f06938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f9f075d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1016 (Thread 0x7f91fa13d640 (LWP 2740950) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fa139750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fa139750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fa139730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fa107be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fa107938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fa1085d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1015 (Thread 0x7f91fa33e640 (LWP 2740949) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fa33a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fa33a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fa33a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fa308be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fa308938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fa3095d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1014 (Thread 0x7f91fa53f640 (LWP 2740948) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fa53b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fa53b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fa53b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fa509be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fa509938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fa50a5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1013 (Thread 0x7f91fa740640 (LWP 2740947) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fa73c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fa73c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fa73c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fa70abe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fa70a938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fa70b5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1012 (Thread 0x7f91fa941640 (LWP 2740946) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fa93d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fa93d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fa93d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fa90bbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fa90b938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fa90c5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1011 (Thread 0x7f91fab42640 (LWP 2740945) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fab3e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fab3e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fab3e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fab0cbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fab0c938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fab0d5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1010 (Thread 0x7f91fad43640 (LWP 2740944) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fad3f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fad3f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fad3f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fad0dbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fad0d938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fad0e5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1009 (Thread 0x7f91faf44640 (LWP 2740943) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91faf40750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91faf40750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91faf40730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91faf0ebe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91faf0e938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91faf0f5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1008 (Thread 0x7f91fb145640 (LWP 2740942) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fb141750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fb141750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fb141730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fb10fbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fb10f938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fb1105d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1007 (Thread 0x7f91fb346640 (LWP 2740941) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fb342750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fb342750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fb342730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fb310be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fb310938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fb3115d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1006 (Thread 0x7f91fb547640 (LWP 2740940) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fb543750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fb543750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fb543730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fb511be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fb511938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fb5125d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1005 (Thread 0x7f91fb748640 (LWP 2740939) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fb744750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fb744750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fb744730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fb712be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fb712938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fb7135d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1004 (Thread 0x7f91fb949640 (LWP 2740938) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fb945750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fb945750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fb945730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fb913be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fb913938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fb9145d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1003 (Thread 0x7f91fbb4a640 (LWP 2740937) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fbb46750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fbb46750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fbb46730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fbb14be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fbb14938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fbb155d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1002 (Thread 0x7f91fbd4b640 (LWP 2740936) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fbd47750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fbd47750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fbd47730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fbd15be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fbd15938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fbd165d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1001 (Thread 0x7f91fbf4c640 (LWP 2740935) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fbf48750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fbf48750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fbf48730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fbf16be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fbf16938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fbf175d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1000 (Thread 0x7f91fc14d640 (LWP 2740934) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fc149750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fc149750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fc149730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fc117be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fc117938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fc1185d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 999 (Thread 0x7f91fc34e640 (LWP 2740933) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fc34a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fc34a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fc34a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fc318be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fc318938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fc3195d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 998 (Thread 0x7f91fc54f640 (LWP 2740932) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fc54b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fc54b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fc54b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fc519be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fc519938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fc51a5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 997 (Thread 0x7f91fc750640 (LWP 2740931) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fc74c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fc74c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fc74c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fc71abe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fc71a938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fc71b5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 996 (Thread 0x7f91fc951640 (LWP 2740930) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fc94d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fc94d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fc94d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fc91bbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fc91b938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fc91c5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 995 (Thread 0x7f91fcb52640 (LWP 2740929) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fcb4e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fcb4e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fcb4e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fcb1cbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fcb1c938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fcb1d5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 994 (Thread 0x7f91fcd53640 (LWP 2740928) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fcd4f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fcd4f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fcd4f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fcd1dbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fcd1d938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fcd1e5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 993 (Thread 0x7f91fcf54640 (LWP 2740927) "sched-worker-pr"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fcf50750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fcf50750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fcf50730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fcf1ebe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fcf1e938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fcf1f5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 992 (Thread 0x7f91fd155640 (LWP 2740922) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fd151750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fd151750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fd151730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fd120b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fd120a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fd121090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 991 (Thread 0x7f91fd356640 (LWP 2740921) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fd352750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fd352750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fd352730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fd321b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fd321a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fd322090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 990 (Thread 0x7f91fd557640 (LWP 2740919) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fd553750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fd553750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fd553730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fd522b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fd522a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fd523090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 989 (Thread 0x7f91fd758640 (LWP 2740917) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fd754750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fd754750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fd754730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fd723b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fd723a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fd724090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 988 (Thread 0x7f91fd959640 (LWP 2740915) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fd955750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fd955750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fd955730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fd924b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fd924a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fd925090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 987 (Thread 0x7f91fdb5a640 (LWP 2740914) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fdb56750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fdb56750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fdb56730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fdb25b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fdb25a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fdb26090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 986 (Thread 0x7f91fdd5b640 (LWP 2740913) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fdd57750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fdd57750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fdd57730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fdd26b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fdd26a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fdd27090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 985 (Thread 0x7f91fdf5c640 (LWP 2740912) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fdf58750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fdf58750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fdf58730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fdf27b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fdf27a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fdf28090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 984 (Thread 0x7f91fe15d640 (LWP 2740911) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fe159750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fe159750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fe159730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fe128b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fe128a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fe129090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 983 (Thread 0x7f91fe35e640 (LWP 2740910) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fe35a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fe35a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fe35a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fe329b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fe329a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fe32a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 982 (Thread 0x7f91fe55f640 (LWP 2740909) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fe55b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fe55b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fe55b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fe52ab30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fe52aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fe52b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 981 (Thread 0x7f91fe760640 (LWP 2740908) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fe75c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fe75c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fe75c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fe72bb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fe72ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fe72c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 980 (Thread 0x7f91fe961640 (LWP 2740907) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fe95d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fe95d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fe95d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fe92cb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fe92ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fe92d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 979 (Thread 0x7f91feb62640 (LWP 2740905) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91feb5e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91feb5e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91feb5e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91feb2db30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91feb2da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91feb2e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 978 (Thread 0x7f91fed63640 (LWP 2740903) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fed5f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fed5f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fed5f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fed2eb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fed2ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fed2f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 977 (Thread 0x7f91fef64640 (LWP 2740901) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fef60750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fef60750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fef60730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fef2fb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fef2fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fef30090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 976 (Thread 0x7f91ff165640 (LWP 2740900) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ff161750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ff161750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ff161730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ff130b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ff130a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ff131090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 975 (Thread 0x7f91ff366640 (LWP 2740899) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ff362750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ff362750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ff362730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ff331b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ff331a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ff332090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 974 (Thread 0x7f91ff567640 (LWP 2740898) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ff563750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ff563750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ff563730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ff532b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ff532a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ff533090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 973 (Thread 0x7f91ff768640 (LWP 2740897) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ff764750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ff764750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ff764730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ff733b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ff733a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ff734090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 972 (Thread 0x7f91ff969640 (LWP 2740896) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ff965750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ff965750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ff965730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ff934b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ff934a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ff935090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 971 (Thread 0x7f91ffb6a640 (LWP 2740895) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ffb66750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ffb66750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ffb66730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ffb35b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ffb35a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ffb36090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 970 (Thread 0x7f91ffd6b640 (LWP 2740894) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ffd67750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ffd67750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ffd67730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ffd36b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ffd36a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ffd37090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 969 (Thread 0x7f91fff6c640 (LWP 2740892) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fff68750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fff68750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fff68730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fff37b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fff37a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fff38090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 968 (Thread 0x7f920016d640 (LWP 2740891) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9200169750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9200169750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9200169730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9200138b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9200138a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9200139090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 967 (Thread 0x7f920036e640 (LWP 2740890) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920036a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920036a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920036a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9200339b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9200339a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920033a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 966 (Thread 0x7f920056f640 (LWP 2740889) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920056b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920056b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920056b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920053ab30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920053aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920053b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 965 (Thread 0x7f9200770640 (LWP 2740888) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920076c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920076c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920076c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920073bb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920073ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920073c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 964 (Thread 0x7f9200971640 (LWP 2740886) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920096d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920096d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920096d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920093cb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920093ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920093d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 963 (Thread 0x7f9200b72640 (LWP 2740885) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9200b6e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9200b6e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9200b6e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9200b3db30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9200b3da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9200b3e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 962 (Thread 0x7f9200d73640 (LWP 2740884) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9200d6f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9200d6f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9200d6f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9200d3eb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9200d3ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9200d3f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 961 (Thread 0x7f9200f74640 (LWP 2740883) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9200f70750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9200f70750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9200f70730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9200f3fb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9200f3fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9200f40090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 960 (Thread 0x7f9201175640 (LWP 2740882) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201171750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201171750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201171730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201140b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201140a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201141090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 959 (Thread 0x7f9201376640 (LWP 2740881) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201372750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201372750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201372730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201341b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201341a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201342090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 958 (Thread 0x7f9201577640 (LWP 2740880) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201573750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201573750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201573730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201542b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201542a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201543090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 957 (Thread 0x7f9201778640 (LWP 2740879) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201774750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201774750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201774730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201743b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201743a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201744090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 956 (Thread 0x7f9201979640 (LWP 2740877) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201975750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201975750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201975730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201944b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201944a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201945090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 955 (Thread 0x7f9201b7a640 (LWP 2740876) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201b76750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201b76750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201b76730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201b45b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201b45a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201b46090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 954 (Thread 0x7f9201d7b640 (LWP 2740875) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201d77750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201d77750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201d77730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201d46b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201d46a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201d47090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 953 (Thread 0x7f9201f7c640 (LWP 2740874) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201f78750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201f78750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201f78730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201f47b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201f47a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201f48090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 952 (Thread 0x7f920217d640 (LWP 2740873) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9202179750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9202179750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9202179730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9202148b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9202148a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9202149090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 951 (Thread 0x7f920237e640 (LWP 2740872) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920237a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920237a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920237a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9202349b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9202349a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920234a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 950 (Thread 0x7f920257f640 (LWP 2740870) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920257b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920257b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920257b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920254ab30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920254aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920254b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 949 (Thread 0x7f9202780640 (LWP 2740869) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920277c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920277c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920277c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920274bb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920274ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920274c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 948 (Thread 0x7f9202981640 (LWP 2740868) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920297d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920297d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920297d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920294cb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920294ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920294d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 947 (Thread 0x7f9202b82640 (LWP 2740867) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9202b7e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9202b7e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9202b7e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9202b4db30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9202b4da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9202b4e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 946 (Thread 0x7f9202d83640 (LWP 2740866) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9202d7f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9202d7f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9202d7f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9202d4eb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9202d4ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9202d4f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 945 (Thread 0x7f9202f84640 (LWP 2740865) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9202f80750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9202f80750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9202f80730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9202f4fb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9202f4fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9202f50090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 944 (Thread 0x7f9203185640 (LWP 2740864) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203181750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203181750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203181730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203150b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203150a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203151090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 943 (Thread 0x7f9203386640 (LWP 2740863) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203382750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203382750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203382730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203351b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203351a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203352090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 942 (Thread 0x7f9203587640 (LWP 2740861) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203583750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203583750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203583730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203552b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203552a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203553090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 941 (Thread 0x7f9203788640 (LWP 2740860) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203784750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203784750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203784730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203753b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203753a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203754090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 940 (Thread 0x7f9203989640 (LWP 2740859) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203985750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203985750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203985730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203954b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203954a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203955090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 939 (Thread 0x7f9203b8a640 (LWP 2740858) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203b86750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203b86750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203b86730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203b55b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203b55a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203b56090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 938 (Thread 0x7f9203d8b640 (LWP 2740857) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203d87750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203d87750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203d87730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203d56b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203d56a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203d57090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 937 (Thread 0x7f9203f8c640 (LWP 2740856) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203f88750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203f88750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203f88730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203f57b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203f57a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203f58090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 936 (Thread 0x7f920418d640 (LWP 2740855) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9204189750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9204189750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9204189730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9204158b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9204158a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9204159090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 935 (Thread 0x7f920438e640 (LWP 2740853) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920438a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920438a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920438a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9204359b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9204359a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920435a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 934 (Thread 0x7f920458f640 (LWP 2740852) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920458b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920458b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920458b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920455ab30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920455aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920455b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 933 (Thread 0x7f9204790640 (LWP 2740851) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920478c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920478c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920478c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920475bb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920475ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920475c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 932 (Thread 0x7f9204991640 (LWP 2740849) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920498d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920498d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920498d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920495cb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920495ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920495d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 931 (Thread 0x7f9204b92640 (LWP 2740848) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9204b8e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9204b8e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9204b8e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9204b5db30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9204b5da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9204b5e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 930 (Thread 0x7f9204d93640 (LWP 2740847) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9204d8f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9204d8f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9204d8f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9204d5eb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9204d5ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9204d5f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 929 (Thread 0x7f9204f94640 (LWP 2740846) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9204f90750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9204f90750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9204f90730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9204f5fb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9204f5fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9204f60090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 928 (Thread 0x7f9205195640 (LWP 2740845) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205191750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205191750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205191730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205160b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205160a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205161090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 927 (Thread 0x7f9205396640 (LWP 2740844) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205392750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205392750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205392730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205361b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205361a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205362090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 926 (Thread 0x7f9205597640 (LWP 2740843) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205593750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205593750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205593730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205562b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205562a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205563090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 925 (Thread 0x7f9205798640 (LWP 2740842) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205794750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205794750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205794730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205763b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205763a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205764090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 924 (Thread 0x7f9205999640 (LWP 2740841) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205995750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205995750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205995730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205964b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205964a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205965090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 923 (Thread 0x7f9205b9a640 (LWP 2740840) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205b96750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205b96750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205b96730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205b65b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205b65a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205b66090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 922 (Thread 0x7f9205d9b640 (LWP 2740839) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205d97750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205d97750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205d97730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205d66b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205d66a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205d67090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 921 (Thread 0x7f9205f9c640 (LWP 2740838) "sched-worker-hi"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205f98750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205f98750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205f98730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205f67b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205f67a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205f68090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 920 (Thread 0x7f920619d640 (LWP 2740836) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9206199750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9206199750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9206199730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9206168b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9206168a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9206169090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 919 (Thread 0x7f920639e640 (LWP 2740835) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920639a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920639a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920639a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9206369b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9206369a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920636a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 918 (Thread 0x7f920659f640 (LWP 2740834) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920659b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920659b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920659b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920656ab30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920656aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920656b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 917 (Thread 0x7f92067a0640 (LWP 2740833) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920679c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920679c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920679c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920676bb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920676ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920676c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 916 (Thread 0x7f92069a1640 (LWP 2740832) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920699d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920699d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920699d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920696cb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920696ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920696d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 915 (Thread 0x7f9206ba2640 (LWP 2740831) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9206b9e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9206b9e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9206b9e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9206b6db30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9206b6da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9206b6e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 914 (Thread 0x7f9206da3640 (LWP 2740830) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9206d9f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9206d9f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9206d9f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9206d6eb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9206d6ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9206d6f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 913 (Thread 0x7f9206fa4640 (LWP 2740829) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9206fa0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9206fa0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9206fa0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9206f6fb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9206f6fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9206f70090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 912 (Thread 0x7f92071a5640 (LWP 2740828) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92071a1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92071a1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92071a1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207170b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207170a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207171090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 911 (Thread 0x7f92073a6640 (LWP 2740827) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92073a2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92073a2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92073a2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207371b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207371a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207372090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 910 (Thread 0x7f92075a7640 (LWP 2740826) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92075a3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92075a3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92075a3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207572b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207572a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207573090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 909 (Thread 0x7f92077a8640 (LWP 2740825) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92077a4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92077a4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92077a4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207773b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207773a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207774090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 908 (Thread 0x7f92079a9640 (LWP 2740824) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92079a5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92079a5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92079a5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207974b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207974a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207975090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 907 (Thread 0x7f9207baa640 (LWP 2740823) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9207ba6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9207ba6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9207ba6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207b75b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207b75a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207b76090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 906 (Thread 0x7f9207dab640 (LWP 2740822) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9207da7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9207da7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9207da7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207d76b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207d76a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207d77090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 905 (Thread 0x7f9207fac640 (LWP 2740821) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9207fa8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9207fa8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9207fa8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207f77b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207f77a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207f78090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 904 (Thread 0x7f92081ad640 (LWP 2740820) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92081a9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92081a9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92081a9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9208178b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9208178a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9208179090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 903 (Thread 0x7f92083ae640 (LWP 2740819) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92083aa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92083aa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92083aa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9208379b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9208379a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920837a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 902 (Thread 0x7f92085af640 (LWP 2740818) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92085ab750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92085ab750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92085ab730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920857ab30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920857aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920857b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 901 (Thread 0x7f92087b0640 (LWP 2740817) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92087ac750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92087ac750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92087ac730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920877bb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920877ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920877c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 900 (Thread 0x7f92089b1640 (LWP 2740816) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92089ad750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92089ad750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92089ad730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920897cb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920897ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920897d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 899 (Thread 0x7f9208bb2640 (LWP 2740815) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9208bae750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9208bae750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9208bae730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9208b7db30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9208b7da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9208b7e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 898 (Thread 0x7f9208db3640 (LWP 2740814) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9208daf750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9208daf750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9208daf730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9208d7eb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9208d7ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9208d7f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 897 (Thread 0x7f9208fb4640 (LWP 2740813) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9208fb0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9208fb0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9208fb0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9208f7fb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9208f7fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9208f80090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 896 (Thread 0x7f92091b5640 (LWP 2740812) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92091b1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92091b1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92091b1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209180b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209180a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209181090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 895 (Thread 0x7f92093b6640 (LWP 2740811) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92093b2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92093b2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92093b2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209381b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209381a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209382090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 894 (Thread 0x7f92095b7640 (LWP 2740810) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92095b3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92095b3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92095b3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209582b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209582a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209583090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 893 (Thread 0x7f92097b8640 (LWP 2740809) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92097b4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92097b4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92097b4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209783b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209783a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209784090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 892 (Thread 0x7f92099b9640 (LWP 2740808) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92099b5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92099b5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92099b5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209984b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209984a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209985090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 891 (Thread 0x7f9209bba640 (LWP 2740807) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9209bb6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9209bb6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9209bb6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209b85b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209b85a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209b86090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 890 (Thread 0x7f9209dbb640 (LWP 2740806) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9209db7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9209db7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9209db7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209d86b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209d86a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209d87090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 889 (Thread 0x7f9209fbc640 (LWP 2740805) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9209fb8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9209fb8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9209fb8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209f87b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209f87a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209f88090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 888 (Thread 0x7f920a1bd640 (LWP 2740804) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920a1b9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920a1b9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920a1b9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920a188b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920a188a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920a189090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 887 (Thread 0x7f920a3be640 (LWP 2740803) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920a3ba750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920a3ba750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920a3ba730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920a389b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920a389a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920a38a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 886 (Thread 0x7f920a5bf640 (LWP 2740802) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920a5bb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920a5bb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920a5bb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920a58ab30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920a58aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920a58b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 885 (Thread 0x7f920a7c0640 (LWP 2740801) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920a7bc750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920a7bc750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920a7bc730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920a78bb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920a78ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920a78c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 884 (Thread 0x7f920a9c1640 (LWP 2740800) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920a9bd750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920a9bd750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920a9bd730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920a98cb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920a98ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920a98d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 883 (Thread 0x7f920abc2640 (LWP 2740799) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920abbe750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920abbe750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920abbe730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920ab8db30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920ab8da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920ab8e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 882 (Thread 0x7f920adc3640 (LWP 2740798) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920adbf750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920adbf750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920adbf730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920ad8eb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920ad8ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920ad8f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 881 (Thread 0x7f920afc4640 (LWP 2740797) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920afc0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920afc0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920afc0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920af8fb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920af8fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920af90090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 880 (Thread 0x7f920b1c5640 (LWP 2740796) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920b1c1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920b1c1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920b1c1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920b190b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920b190a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920b191090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 879 (Thread 0x7f920b3c6640 (LWP 2740795) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920b3c2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920b3c2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920b3c2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920b391b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920b391a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920b392090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 878 (Thread 0x7f920b5c7640 (LWP 2740794) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920b5c3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920b5c3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920b5c3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920b592b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920b592a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920b593090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 877 (Thread 0x7f920b7c8640 (LWP 2740793) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920b7c4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920b7c4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920b7c4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920b793b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920b793a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920b794090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 876 (Thread 0x7f920b9c9640 (LWP 2740792) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920b9c5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920b9c5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920b9c5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920b994b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920b994a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920b995090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 875 (Thread 0x7f920bbca640 (LWP 2740791) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920bbc6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920bbc6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920bbc6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920bb95b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920bb95a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920bb96090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 874 (Thread 0x7f920bdcb640 (LWP 2740790) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920bdc7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920bdc7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920bdc7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920bd96b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920bd96a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920bd97090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 873 (Thread 0x7f920bfcc640 (LWP 2740789) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920bfc8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920bfc8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920bfc8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920bf97b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920bf97a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920bf98090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 872 (Thread 0x7f920c1cd640 (LWP 2740788) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920c1c9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920c1c9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920c1c9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920c198b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920c198a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920c199090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 871 (Thread 0x7f920c3ce640 (LWP 2740787) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920c3ca750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920c3ca750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920c3ca730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920c399b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920c399a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920c39a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 870 (Thread 0x7f920c5cf640 (LWP 2740786) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920c5cb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920c5cb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920c5cb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920c59ab30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920c59aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920c59b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 869 (Thread 0x7f920c7d0640 (LWP 2740785) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920c7cc750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920c7cc750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920c7cc730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920c79bb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920c79ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920c79c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 868 (Thread 0x7f920c9d1640 (LWP 2740784) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920c9cd750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920c9cd750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920c9cd730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920c99cb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920c99ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920c99d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 867 (Thread 0x7f920cbd2640 (LWP 2740783) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920cbce750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920cbce750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920cbce730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920cb9db30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920cb9da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920cb9e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 866 (Thread 0x7f920cdd3640 (LWP 2740782) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920cdcf750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920cdcf750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920cdcf730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920cd9eb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920cd9ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920cd9f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 865 (Thread 0x7f920cfd4640 (LWP 2740781) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920cfd0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920cfd0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920cfd0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920cf9fb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920cf9fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920cfa0090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 864 (Thread 0x7f920d1d5640 (LWP 2740780) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920d1d1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920d1d1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920d1d1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920d1a0b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920d1a0a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920d1a1090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 863 (Thread 0x7f920d3d6640 (LWP 2740779) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920d3d2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920d3d2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920d3d2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920d3a1b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920d3a1a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920d3a2090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 862 (Thread 0x7f920d5d7640 (LWP 2740778) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920d5d3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920d5d3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920d5d3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920d5a2b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920d5a2a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920d5a3090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 861 (Thread 0x7f920d7d8640 (LWP 2740777) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920d7d4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920d7d4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920d7d4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920d7a3b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920d7a3a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920d7a4090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 860 (Thread 0x7f920d9d9640 (LWP 2740776) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920d9d5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920d9d5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920d9d5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920d9a4b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920d9a4a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920d9a5090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 859 (Thread 0x7f920dbda640 (LWP 2740775) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920dbd6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920dbd6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920dbd6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920dba5b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920dba5a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920dba6090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 858 (Thread 0x7f920dddb640 (LWP 2740774) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920ddd7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920ddd7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920ddd7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920dda6b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920dda6a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920dda7090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 857 (Thread 0x7f920dfdc640 (LWP 2740773) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920dfd8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920dfd8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920dfd8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920dfa7b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920dfa7a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920dfa8090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 856 (Thread 0x7f920e1dd640 (LWP 2740772) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920e1d9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920e1d9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920e1d9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920e1a8b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920e1a8a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920e1a9090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 855 (Thread 0x7f920e3de640 (LWP 2740771) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920e3da750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920e3da750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920e3da730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920e3a9b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920e3a9a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920e3aa090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 854 (Thread 0x7f920e5df640 (LWP 2740770) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920e5db750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920e5db750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920e5db730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920e5aab30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920e5aaa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920e5ab090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 853 (Thread 0x7f920e7e0640 (LWP 2740769) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920e7dc750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920e7dc750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920e7dc730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920e7abb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920e7aba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920e7ac090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 852 (Thread 0x7f920e9e1640 (LWP 2740768) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920e9dd750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920e9dd750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920e9dd730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920e9acb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920e9aca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920e9ad090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 851 (Thread 0x7f920ebe2640 (LWP 2740767) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920ebde750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920ebde750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920ebde730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920ebadb30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920ebada10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920ebae090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 850 (Thread 0x7f94afffe640 (LWP 2740766) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94afffa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94afffa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94afffa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94affc9b30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f94affc9a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f94affca090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 849 (Thread 0x7f94b19ff640 (LWP 2740765) "sched-worker-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94b19fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94b19fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94b19fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94b19cab30) at src/pool/spawn.rs:305 +#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f94b19caa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f94b19cb090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 848 (Thread 0x7f94b21ff640 (LWP 2740759) "resource-meteri"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94b21fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94b21fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94b21fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689433552, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94b21cac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94b21cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94b21cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 847 (Thread 0x7f94b3dff640 (LWP 2740758) "resource-meteri"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94b3dfb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94b3dfb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94b3dfb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140277123436240, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94b3dcac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94b3dcab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94b3dcb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 846 (Thread 0x7f94b55ff640 (LWP 2740757) "resource-meteri"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94b55fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94b55fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94b55fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247604720, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94b55cac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94b55cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94b55cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 845 (Thread 0x7f94b77ff640 (LWP 2740756) "debugger"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94b77fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94b77fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f94b77fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park (key=140279689410680, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f95690e7878, mutex=0x7f95690e7880, timeout=...) at src/condvar.rs:333 +#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f95690e7878, mutex_guard=0x7f94b77ca118) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 +#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f95690e7878, guard=...) at src/loom/std/parking_lot.rs:149 +#9 0x00007f95830ef2f6 in tokio::runtime::park::Inner::park (self=0x7f95690e7870) at src/runtime/park.rs:116 +#10 0x00007f95830ef113 in tokio::runtime::park::ParkThread::park (self=0x7f9561594ea0) at src/runtime/park.rs:62 +#11 0x00007f95830f1adf in tokio::runtime::driver::IoStack::park (self=0x7f9561594e98, handle=0x7f9561584d18) at src/runtime/driver.rs:176 +#12 0x00007f958316a02e in tokio::runtime::time::Driver::park_internal (self=0x7f9561594e98, rt_handle=0x7f9561584d18, limit=...) at src/runtime/time/mod.rs:235 +#13 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f9561594e98, handle=0x7f9561584d18) at src/runtime/time/mod.rs:165 +#14 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f9561594e90, handle=0x7f9561584d18) at src/runtime/driver.rs:332 +#15 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f9561594e90, handle=0x7f9561584d18) at src/runtime/driver.rs:71 +#16 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f95690e7930, driver=0x7f9561594e90, handle=0x7f9561584d18) at src/runtime/scheduler/multi_thread/park.rs:184 +#17 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f95690e7930, handle=0x7f9561584d18) at src/runtime/scheduler/multi_thread/park.rs:117 +#18 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f94b77ca9a0, handle=0x7f9561584d18) at src/runtime/scheduler/multi_thread/park.rs:67 +#19 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f94b77cae98, core=0x7f9561598f90, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 +#20 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f94b77cae98, core=0x7f9561598f90) at src/runtime/scheduler/multi_thread/worker.rs:702 +#21 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f94b77cae98, core=0x7f9561598f90) at src/runtime/scheduler/multi_thread/worker.rs:553 +#22 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 +#23 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f94b77cc610, t=0x7f94b77cae90, f=...) at src/runtime/context/scoped.rs:40 +#24 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f94b77cc5d8) at src/runtime/context.rs:180 +#25 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#26 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#27 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f94b77cae90, f=...) at src/runtime/context.rs:180 +#28 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 +#29 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f94b77cb098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 +#30 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 +#31 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 +#32 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f94b77cb220) at src/runtime/blocking/task.rs:42 +#33 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594e28) at src/runtime/task/core.rs:328 +#34 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594e28, f=...) at src/loom/std/unsafe_cell.rs:16 +#35 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594e20, cx=...) at src/runtime/task/core.rs:317 +#36 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 +#37 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#38 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f94b77cb398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#39 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#40 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#41 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#42 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594e20, cx=...) at src/runtime/task/harness.rs:473 +#43 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f94b77cb5b0) at src/runtime/task/harness.rs:208 +#44 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 +#45 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 +#46 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 +#47 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 +#48 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 +#49 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f95615accd0, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 +#50 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 +#51 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#52 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#53 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#54 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f94b77cba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#55 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#56 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#57 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#58 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#59 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#60 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#61 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#62 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#63 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#64 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 844 (Thread 0x7f92145e3640 (LWP 2740755) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92145df750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92145df750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92145df730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92145adbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92145ad938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92145ae5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 843 (Thread 0x7f9214fe4640 (LWP 2740754) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9214fe0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9214fe0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9214fe0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9214faebe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9214fae938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9214faf5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 842 (Thread 0x7f92159e5640 (LWP 2740753) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92159e1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92159e1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92159e1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92159afbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92159af938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92159b05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 841 (Thread 0x7f92163e6640 (LWP 2740752) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92163e2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92163e2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92163e2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92163b0be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92163b0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92163b15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 840 (Thread 0x7f9216de7640 (LWP 2740751) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9216de3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9216de3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9216de3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9216db1be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9216db1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9216db25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 839 (Thread 0x7f92177e8640 (LWP 2740750) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92177e4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92177e4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92177e4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92177b2be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92177b2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92177b35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 838 (Thread 0x7f92181e9640 (LWP 2740749) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92181e5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92181e5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92181e5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92181b3be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92181b3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92181b45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 837 (Thread 0x7f9218bea640 (LWP 2740748) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9218be6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9218be6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9218be6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9218bb4be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9218bb4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9218bb55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 836 (Thread 0x7f92195eb640 (LWP 2740747) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92195e7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92195e7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92195e7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92195b5be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92195b5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92195b65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 835 (Thread 0x7f9219fec640 (LWP 2740746) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9219fe8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9219fe8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9219fe8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9219fb6be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9219fb6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9219fb75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 834 (Thread 0x7f921a9ed640 (LWP 2740745) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921a9e9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921a9e9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921a9e9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921a9b7be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921a9b7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921a9b85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 833 (Thread 0x7f921b3ee640 (LWP 2740744) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921b3ea750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921b3ea750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921b3ea730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921b3b8be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921b3b8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921b3b95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 832 (Thread 0x7f921bdef640 (LWP 2740743) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921bdeb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921bdeb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921bdeb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921bdb9be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921bdb9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921bdba5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 831 (Thread 0x7f921c7f0640 (LWP 2740742) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921c7ec750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921c7ec750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921c7ec730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921c7babe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921c7ba938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921c7bb5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 830 (Thread 0x7f921d1f1640 (LWP 2740741) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921d1ed750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921d1ed750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921d1ed730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921d1bbbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921d1bb938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921d1bc5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 829 (Thread 0x7f921dbf2640 (LWP 2740740) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921dbee750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921dbee750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921dbee730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921dbbcbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921dbbc938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921dbbd5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 828 (Thread 0x7f921e5f3640 (LWP 2740739) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921e5ef750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921e5ef750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921e5ef730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921e5bdbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921e5bd938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921e5be5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 827 (Thread 0x7f921eff4640 (LWP 2740738) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921eff0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921eff0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921eff0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921efbebe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921efbe938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921efbf5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 826 (Thread 0x7f921f9f5640 (LWP 2740737) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921f9f1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921f9f1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921f9f1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921f9bfbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921f9bf938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921f9c05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 825 (Thread 0x7f92203f6640 (LWP 2740736) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92203f2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92203f2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92203f2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92203c0be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92203c0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92203c15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 824 (Thread 0x7f9220df7640 (LWP 2740735) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9220df3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9220df3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9220df3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9220dc1be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9220dc1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9220dc25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 823 (Thread 0x7f92217f8640 (LWP 2740734) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92217f4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92217f4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92217f4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92217c2be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92217c2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92217c35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 822 (Thread 0x7f92221f9640 (LWP 2740733) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92221f5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92221f5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92221f5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92221c3be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92221c3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92221c45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 821 (Thread 0x7f9222bfa640 (LWP 2740732) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9222bf6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9222bf6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9222bf6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9222bc4be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9222bc4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9222bc55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 820 (Thread 0x7f92235fb640 (LWP 2740731) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92235f7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92235f7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92235f7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92235c5be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92235c5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92235c65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 819 (Thread 0x7f9223ffc640 (LWP 2740730) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9223ff8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9223ff8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9223ff8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9223fc6be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9223fc6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9223fc75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 818 (Thread 0x7f92249fd640 (LWP 2740729) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92249f9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92249f9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92249f9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92249c7be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92249c7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92249c85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 817 (Thread 0x7f92253fe640 (LWP 2740728) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92253fa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92253fa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92253fa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92253c8be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92253c8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92253c95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 816 (Thread 0x7f9225dff640 (LWP 2740727) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9225dfb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9225dfb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9225dfb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9225dc9be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9225dc9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9225dca5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 815 (Thread 0x7f9226800640 (LWP 2740726) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92267fc750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92267fc750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92267fc730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92267cabe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92267ca938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92267cb5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 814 (Thread 0x7f9227201640 (LWP 2740725) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92271fd750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92271fd750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92271fd730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92271cbbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92271cb938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92271cc5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 813 (Thread 0x7f9227c02640 (LWP 2740724) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9227bfe750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9227bfe750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9227bfe730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9227bccbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9227bcc938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9227bcd5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 812 (Thread 0x7f9228603640 (LWP 2740723) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92285ff750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92285ff750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92285ff730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92285cdbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92285cd938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92285ce5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 811 (Thread 0x7f9229004640 (LWP 2740722) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9229000750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9229000750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9229000730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9228fcebe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9228fce938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9228fcf5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 810 (Thread 0x7f9229a05640 (LWP 2740721) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9229a01750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9229a01750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9229a01730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92299cfbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92299cf938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92299d05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 809 (Thread 0x7f922a406640 (LWP 2740720) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922a402750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922a402750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922a402730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922a3d0be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922a3d0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922a3d15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 808 (Thread 0x7f922ae07640 (LWP 2740719) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922ae03750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922ae03750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922ae03730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922add1be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922add1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922add25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 807 (Thread 0x7f922b808640 (LWP 2740718) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922b804750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922b804750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922b804730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922b7d2be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922b7d2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922b7d35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 806 (Thread 0x7f922c209640 (LWP 2740717) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922c205750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922c205750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922c205730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922c1d3be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922c1d3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922c1d45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 805 (Thread 0x7f922cc0a640 (LWP 2740716) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922cc06750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922cc06750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922cc06730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922cbd4be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922cbd4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922cbd55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 804 (Thread 0x7f922d60b640 (LWP 2740715) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922d607750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922d607750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922d607730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922d5d5be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922d5d5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922d5d65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 803 (Thread 0x7f922e00c640 (LWP 2740714) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922e008750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922e008750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922e008730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922dfd6be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922dfd6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922dfd75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 802 (Thread 0x7f922ea0d640 (LWP 2740713) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922ea09750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922ea09750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922ea09730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922e9d7be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922e9d7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922e9d85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 801 (Thread 0x7f922f40e640 (LWP 2740712) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922f40a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922f40a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922f40a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922f3d8be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922f3d8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922f3d95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 800 (Thread 0x7f922fe0f640 (LWP 2740711) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922fe0b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922fe0b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922fe0b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922fdd9be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922fdd9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922fdda5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 799 (Thread 0x7f9230810640 (LWP 2740710) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923080c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923080c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923080c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92307dabe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92307da938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92307db5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 798 (Thread 0x7f9231211640 (LWP 2740709) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923120d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923120d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923120d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92311dbbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92311db938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92311dc5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 797 (Thread 0x7f9231c12640 (LWP 2740708) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9231c0e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9231c0e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9231c0e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9231bdcbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9231bdc938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9231bdd5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 796 (Thread 0x7f9232613640 (LWP 2740707) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923260f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923260f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923260f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92325ddbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92325dd938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92325de5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 795 (Thread 0x7f9233014640 (LWP 2740706) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9233010750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9233010750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9233010730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9232fdebe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9232fde938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9232fdf5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 794 (Thread 0x7f9233a15640 (LWP 2740705) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9233a11750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9233a11750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9233a11730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92339dfbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92339df938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92339e05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 793 (Thread 0x7f9234416640 (LWP 2740704) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9234412750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9234412750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9234412730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92343e0be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92343e0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92343e15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 792 (Thread 0x7f9234e17640 (LWP 2740703) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9234e13750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9234e13750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9234e13730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9234de1be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9234de1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9234de25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 791 (Thread 0x7f9235818640 (LWP 2740702) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9235814750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9235814750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9235814730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92357e2be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92357e2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92357e35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 790 (Thread 0x7f9236219640 (LWP 2740701) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9236215750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9236215750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9236215730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92361e3be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92361e3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92361e45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 789 (Thread 0x7f9236c1a640 (LWP 2740700) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9236c16750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9236c16750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9236c16730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9236be4be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9236be4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9236be55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 788 (Thread 0x7f923761b640 (LWP 2740699) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9237617750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9237617750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9237617730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92375e5be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92375e5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92375e65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 787 (Thread 0x7f923801c640 (LWP 2740698) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9238018750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9238018750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9238018730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9237fe6be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9237fe6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9237fe75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 786 (Thread 0x7f9238a1d640 (LWP 2740697) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9238a19750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9238a19750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9238a19730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92389e7be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92389e7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92389e85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 785 (Thread 0x7f923941e640 (LWP 2740696) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923941a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923941a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923941a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92393e8be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92393e8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92393e95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 784 (Thread 0x7f9239e1f640 (LWP 2740695) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9239e1b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9239e1b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9239e1b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9239de9be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9239de9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9239dea5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 783 (Thread 0x7f923c623640 (LWP 2740694) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923c61f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923c61f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923c61f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923c5edbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923c5ed938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923c5ee5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 782 (Thread 0x7f923d024640 (LWP 2740693) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923d020750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923d020750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923d020730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923cfeebe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923cfee938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923cfef5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 781 (Thread 0x7f923da25640 (LWP 2740692) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923da21750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923da21750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923da21730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923d9efbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923d9ef938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923d9f05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 780 (Thread 0x7f923e426640 (LWP 2740691) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923e422750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923e422750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923e422730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923e3f0be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923e3f0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923e3f15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 779 (Thread 0x7f923ee27640 (LWP 2740690) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923ee23750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923ee23750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923ee23730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923edf1be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923edf1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923edf25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 778 (Thread 0x7f923f828640 (LWP 2740689) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923f824750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923f824750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923f824730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923f7f2be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923f7f2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923f7f35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 777 (Thread 0x7f9240229640 (LWP 2740688) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9240225750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9240225750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9240225730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92401f3be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92401f3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92401f45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 776 (Thread 0x7f9240c2a640 (LWP 2740687) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9240c26750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9240c26750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9240c26730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9240bf4be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9240bf4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9240bf55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 775 (Thread 0x7f924162b640 (LWP 2740686) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9241627750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9241627750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9241627730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92415f5be0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92415f5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92415f65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 774 (Thread 0x7f923bc22640 (LWP 2740685) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923bc1e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923bc1e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923bc1e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923bbecbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923bbec938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923bbed5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 773 (Thread 0x7f923b221640 (LWP 2740684) "unified-read-po"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923b21d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923b21d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923b21d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923b1ebbe0) at src/pool/spawn.rs:305 +#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923b1eb938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923b1ec5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 772 (Thread 0x7f94bc3fe640 (LWP 2740683) "pd-worker-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94bc3fa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94bc3fa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94bc3fa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689434352, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94bc3c9c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94bc3c9b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94bc3ca170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 771 (Thread 0x7f94c29ff640 (LWP 2740681) "gc-worker-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94c29fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94c29fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94c29fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140277122565072, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94c29cac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94c29cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94c29cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 770 (Thread 0x7f94c61ff640 (LWP 2740680) "flow-checker"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cfc2f4 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 0x00007f9583cea9cc in std::sys_common::thread_parking::futex::Parker::park_timeout () at library/std/src/sys_common/thread_parking/futex.rs:72 +#3 std::thread::park_timeout () at library/std/src/thread/mod.rs:1133 +#4 0x00007f957e13f3e0 in std::sync::mpmc::context::Context::wait_until (self=0x7f94c61ca498, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:130 +#5 0x00007f957e42fa4a in std::sync::mpmc::list::{impl#3}::recv::{closure#1} (cx=0x7f94c61ca498) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:444 +#6 0x00007f957e4349d0 in std::sync::mpmc::context::{impl#0}::with::{closure#0}, ()> (cx=, cx=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:50 +#7 std::sync::mpmc::context::{impl#0}::with::{closure#1}, ()> (cell=0x7f94c61fba38) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:58 +#8 0x00007f957e439c1b in std::thread::local::LocalKey>>::try_with>, std::sync::mpmc::context::{impl#0}::with::{closure_env#1}, ()>, ()> (self=0x7f9583da0378, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f957e4316de in std::sync::mpmc::context::Context::with, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:53 +#10 0x00007f957e42f4b1 in std::sync::mpmc::list::Channel::recv (self=0x7f956a81e600, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:434 +#11 0x00007f957e13e8fc in std::sync::mpmc::Receiver::recv_deadline (self=0x7f94c61cade0, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/mod.rs:340 +#12 0x00007f957dfe1ad0 in std::sync::mpsc::Receiver::recv_deadline (self=0x7f94c61cade0, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpsc/mod.rs:970 +#13 0x00007f957d07b0e3 in tikv::storage::txn::flow_controller::singleton_flow_controller::{impl#6}::start::{closure#0} () at src/storage/txn/flow_controller/singleton_flow_controller.rs:489 +#14 0x00007f957df427c9 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, ()> () at components/tikv_util/src/sys/thread.rs:441 +#15 0x00007f957a80ae97 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#16 0x00007f957cb7a790 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#17 0x00007f957c9e2b04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#18 0x00007f957bdee664 in std::panicking::try::do_call, ()>, ()>>, ()> (data=0x7f94c61cb250) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#19 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#20 0x00007f957bdaee78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#21 0x00007f957cb79a00 in std::panic::catch_unwind, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#22 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#23 0x00007f957d67eb9f in core::ops::function::FnOnce::call_once, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#24 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#25 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#26 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#27 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#28 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 769 (Thread 0x7f923a820640 (LWP 2740679) "RaftStoreProxy"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f957068996f in pthread_cond_timedwait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9582217915 in rocksdb::port::CondVar::TimedWait(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f95823a23e9 in rocksdb::InstrumentedCondVar::TimedWait(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f95825f0201 in rocksdb::Timer::Run() () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f95825f06f2 in void* std::__1::__thread_proxy[abi:ue170006] >, void (rocksdb::Timer::*)(), rocksdb::Timer*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 768 (Thread 0x7f924202c640 (LWP 2740611) "rocksdb:high"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 767 (Thread 0x7f9242a2d640 (LWP 2740610) "rocksdb:high"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 766 (Thread 0x7f924342e640 (LWP 2740609) "rocksdb:high"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 765 (Thread 0x7f9243e2f640 (LWP 2740608) "rocksdb:low"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 764 (Thread 0x7f9244830640 (LWP 2740607) "rocksdb:low"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 763 (Thread 0x7f9245231640 (LWP 2740606) "rocksdb:low"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 762 (Thread 0x7f9245c32640 (LWP 2740605) "rocksdb:low"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 761 (Thread 0x7f9246633640 (LWP 2740604) "rocksdb:low"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 760 (Thread 0x7f9247034640 (LWP 2740603) "rocksdb:low"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 759 (Thread 0x7f94ad3ff640 (LWP 2740595) "sst-recovery-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94ad3fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94ad3fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94ad3fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140277123170256, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94ad3cac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94ad3cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94ad3cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 758 (Thread 0x7f94cd7ff640 (LWP 2740592) "region-collecto"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94cd7fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94cd7fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94cd7fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140277106202736, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94cd7cac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94cd7cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94cd7cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 757 (Thread 0x7f95614ff640 (LWP 2740591) "re-metrics"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cfc2f4 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 0x00007f9583cea9cc in std::sys_common::thread_parking::futex::Parker::park_timeout () at library/std/src/sys_common/thread_parking/futex.rs:72 +#3 std::thread::park_timeout () at library/std/src/thread/mod.rs:1133 +#4 0x00007f95806bd330 in std::sync::mpmc::context::Context::wait_until (self=0x7f95614cb2c8, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:130 +#5 0x00007f9580705c4a in std::sync::mpmc::list::{impl#3}::recv::{closure#1}<()> (cx=0x7f95614cb2c8) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:444 +#6 0x00007f95806bd940 in std::sync::mpmc::context::{impl#0}::with::{closure#0}, ()> (cx=, cx=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:50 +#7 std::sync::mpmc::context::{impl#0}::with::{closure#1}, ()> (cell=0x7f95614fba38) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:58 +#8 0x00007f95807089db in std::thread::local::LocalKey>>::try_with>, std::sync::mpmc::context::{impl#0}::with::{closure_env#1}, ()>, ()> (self=0x7f9583e05508, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f95806bd44e in std::sync::mpmc::context::Context::with, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:53 +#10 0x00007f9580705b25 in std::sync::mpmc::list::Channel<()>::recv<()> (self=0x7f956a81ea00, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:434 +#11 0x00007f95806a480c in std::sync::mpmc::Receiver<()>::recv_deadline<()> (self=0x7f95614cb7a0, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/mod.rs:340 +#12 0x00007f95806a471a in std::sync::mpmc::Receiver<()>::recv_timeout<()> (self=0x7f95614cb7a0, timeout=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/mod.rs:323 +#13 0x00007f95806e6ea7 in std::sync::mpsc::Receiver<()>::recv_timeout<()> (self=0x7f95614cb7a0, timeout=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpsc/mod.rs:909 +#14 0x00007f95806547ea in raft_engine::engine::{impl#1}::open_with::{closure#0} () at /home/xzx/.cargo/git/checkouts/raft-engine-35ec7b0b2c07ddd2/de1ec93/src/engine.rs:112 +#15 0x00007f9580690607 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#16 0x00007f95806847fd in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#17 0x00007f958066a9a1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#18 0x00007f958063f2c5 in std::panicking::try::do_call, ()>>, ()> (data=0x7f95614cb900) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#19 0x00007f958064db1b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#20 0x00007f958063ead2 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#21 0x00007f9580684629 in std::panic::catch_unwind, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#22 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#23 0x00007f958065d12f in core::ops::function::FnOnce::call_once, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#24 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#25 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#26 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#27 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#28 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 756 (Thread 0x7f9273635640 (LWP 2740419) "SessionCleaner"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7ffd1e7b6408, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7ffd1e7b6408, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c23628f29 in std::__1::condition_variable::wait_until >, DB::SessionCleaner::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::SessionCleaner::run()::$_0) (this=0x7ffd1e7b6408, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c23628ecf in std::__1::condition_variable::wait_for[abi:ue170006], DB::SessionCleaner::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::SessionCleaner::run()::$_0) (this=0x7ffd1e7b6408, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 +#6 0x0000558c23628e13 in DB::SessionCleaner::run (this=0x7ffd1e7b63d8) at /DATA/disk3/xzx/tiflash/dbms/src/Interpreters/Context.cpp:2131 +#7 0x0000558c1ab89b49 in std::__1::__invoke[abi:ue170006](void (DB::SessionCleaner::*&&)(), DB::SessionCleaner*&&) (__f=@0x7f95702ba9a8: (void (DB::SessionCleaner::*)(class DB::SessionCleaner * const)) 0x558c23628da0 , __a0=@0x7f95702ba9b8: 0x7ffd1e7b63d8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#8 0x0000558c1ab89ace in std::__1::__thread_execute[abi:ue170006] >, void (DB::SessionCleaner::*)(), DB::SessionCleaner*, 2ul>(std::__1::tuple >, void (DB::SessionCleaner::*)(), DB::SessionCleaner*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c1ab898a2 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::SessionCleaner::*)(), DB::SessionCleaner*> >(void*) (__vp=0x7f95702ba9a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 755 (Thread 0x7f9274036640 (LWP 2740418) "civetweb-master"): +#0 0x00007f9570701fdf in poll () from /lib64/libc.so.6 +#1 0x0000558c27835555 in mg_poll (pfd=0x7f940afd42a8, n=1, milliseconds=2000, stop_flag=0x7f95702d7028) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:5936 +#2 0x0000558c2783e895 in master_thread_run (ctx=0x7f95702d7000) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:19236 +#3 0x0000558c278329bc in master_thread (thread_func_param=0x7f95702d7000) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:19347 +#4 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#5 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 754 (Thread 0x7f9274a37640 (LWP 2740417) "civetweb-worker"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x0000558c2783b243 in consume_socket (ctx=0x7f95702d7000, sp=0x7f92b63b52e0, thread_index=1) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:18740 +#3 0x0000558c2783af61 in worker_thread_run (conn=0x7f92b63b49e8) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:18881 +#4 0x0000558c2783295c in worker_thread (thread_func_param=0x7f92b63b49e8) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:19054 +#5 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#6 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 753 (Thread 0x7f9275438640 (LWP 2740416) "civetweb-worker"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x0000558c2783b243 in consume_socket (ctx=0x7f95702d7000, sp=0x7f92b63b48f8, thread_index=0) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:18740 +#3 0x0000558c2783af61 in worker_thread_run (conn=0x7f92b63b4000) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:18881 +#4 0x0000558c2783295c in worker_thread (thread_func_param=0x7f92b63b4000) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:19054 +#5 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#6 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 752 (Thread 0x7f9275e39640 (LWP 2740415) "Prometheus"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f95702b0220, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f95702b0220, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c27b7a0f9 in std::__1::condition_variable::wait_until >, Poco::Event::waitImpl(long)::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, Poco::Event::waitImpl(long)::$_0) (this=0x7f95702b0220, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c27b7a06f in std::__1::condition_variable::wait_for[abi:ue170006], Poco::Event::waitImpl(long)::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, Poco::Event::waitImpl(long)::$_0) (this=0x7f95702b0220, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 +#6 0x0000558c27b79e98 in Poco::Event::waitImpl (this=0x7f95702b0220, milliseconds=14999) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Event.cpp:94 +#7 0x0000558c1ac82fed in Poco::Event::tryWait (this=0x7f95702b0220, milliseconds=14999) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/include/Poco/Event.h:109 +#8 0x0000558c27c0173b in Poco::TimedNotificationQueue::wait (this=0x7f95702b0208, interval=14998891) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/TimedNotificationQueue.cpp:165 +#9 0x0000558c27c014ee in Poco::TimedNotificationQueue::waitDequeueNotification (this=0x7f95702b0208) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/TimedNotificationQueue.cpp:99 +#10 0x0000558c27a63e9f in Poco::Util::Timer::run (this=0x7f95702b0200) at /DATA/disk3/xzx/tiflash/contrib/poco/Util/src/Timer.cpp:330 +#11 0x0000558c1ac3fd69 in DB::Timer::run (this=0x7f95702b0200) at /DATA/disk3/xzx/tiflash/dbms/src/Common/Timer.h:32 +#12 0x0000558c27bf81ea in Poco::(anonymous namespace)::RunnableHolder::run (this=0x7f94936ee610) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread.cpp:43 +#13 0x0000558c27bf72ea in Poco::ThreadImpl::runnableEntry (pThread=0x7f95702b02e0) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread_STD.cpp:139 +#14 0x0000558c27bf9760 in std::__1::__invoke[abi:ue170006](void* (*&&)(void*), Poco::ThreadImpl*&&) (__f=@0x7f95702a6a48: 0x558c27bf7280 , __args=@0x7f95702a6a50: 0x7f95702b02e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#15 0x0000558c27bf972e in std::__1::__thread_execute[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*, 2ul>(std::__1::tuple >, void* (*)(void*), Poco::ThreadImpl*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#16 0x0000558c27bf9512 in std::__1::__thread_proxy[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*> >(void*) (__vp=0x7f95702a6a40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#17 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#18 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 751 (Thread 0x7f927683a640 (LWP 2740414) "AsyncMetrics"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7ffd1e7b64b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c23619977 in std::__1::condition_variable::wait_until > > (this=0x7ffd1e7b64b0, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c236119a9 in std::__1::condition_variable::wait_until >, DB::AsynchronousMetrics::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::AsynchronousMetrics::run()::$_0) (this=0x7ffd1e7b64b0, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c23611855 in DB::AsynchronousMetrics::run (this=0x7ffd1e7b6478) at /DATA/disk3/xzx/tiflash/dbms/src/Interpreters/AsynchronousMetrics.cpp:100 +#6 0x0000558c1ab88e38 in DB::AsynchronousMetrics::AsynchronousMetrics(DB::Context&)::{lambda()#1}::operator()() const (this=0x7f94936ee608) at /DATA/disk3/xzx/tiflash/dbms/src/Interpreters/AsynchronousMetrics.h:40 +#7 0x0000558c1ab88df5 in std::__1::__invoke[abi:ue170006](DB::AsynchronousMetrics::AsynchronousMetrics(DB::Context&)::{lambda()#1}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c1ab88d8d in std::__1::__thread_execute[abi:ue170006] >, DB::AsynchronousMetrics::AsynchronousMetrics(DB::Context&)::{lambda()#1}>(std::__1::tuple >, DB::AsynchronousMetrics::AsynchronousMetrics(DB::Context&)::{lambda()#1}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c1ab88942 in std::__1::__thread_proxy[abi:ue170006] >, DB::AsynchronousMetrics::AsynchronousMetrics(DB::Context&)::{lambda()#1}> >(void*) (__vp=0x7f94936ee600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 750 (Thread 0x7f927723b640 (LWP 2740413) "UserCfgReloader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f9570295598, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f9570295598, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c26be9449 in std::__1::condition_variable::wait_until >, DB::ConfigReloader::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::ConfigReloader::run()::$_0) (this=0x7f9570295598, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c26be934f in std::__1::condition_variable::wait_for[abi:ue170006], DB::ConfigReloader::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::ConfigReloader::run()::$_0) (this=0x7f9570295598, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 +#6 0x0000558c26be90d8 in DB::ConfigReloader::run (this=0x7f95702954c0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/Config/ConfigReloader.cpp:67 +#7 0x0000558c26bea279 in std::__1::__invoke[abi:ue170006](void (DB::ConfigReloader::*&&)(), DB::ConfigReloader*&&) (__f=@0x7f95702a69e8: (void (DB::ConfigReloader::*)(class DB::ConfigReloader * const)) 0x558c26be9080 , __a0=@0x7f95702a69f8: 0x7f95702954c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#8 0x0000558c26bea1fe in std::__1::__thread_execute[abi:ue170006] >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*, 2ul>(std::__1::tuple >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c26be9fd2 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*> >(void*) (__vp=0x7f95702a69e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 749 (Thread 0x7f927883c640 (LWP 2740412) "CfgReloader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09dd58, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f956f09dd58, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c26be9449 in std::__1::condition_variable::wait_until >, DB::ConfigReloader::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::ConfigReloader::run()::$_0) (this=0x7f956f09dd58, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c26be934f in std::__1::condition_variable::wait_for[abi:ue170006], DB::ConfigReloader::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::ConfigReloader::run()::$_0) (this=0x7f956f09dd58, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 +#6 0x0000558c26be90d8 in DB::ConfigReloader::run (this=0x7f956f09dc80) at /DATA/disk3/xzx/tiflash/dbms/src/Common/Config/ConfigReloader.cpp:67 +#7 0x0000558c26bea279 in std::__1::__invoke[abi:ue170006](void (DB::ConfigReloader::*&&)(), DB::ConfigReloader*&&) (__f=@0x7f95702b77c8: (void (DB::ConfigReloader::*)(class DB::ConfigReloader * const)) 0x558c26be9080 , __a0=@0x7f95702b77d8: 0x7f956f09dc80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#8 0x0000558c26bea1fe in std::__1::__thread_execute[abi:ue170006] >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*, 2ul>(std::__1::tuple >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c26be9fd2 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*> >(void*) (__vp=0x7f95702b77c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 748 (Thread 0x7f927923d640 (LWP 2740411) "TiFlashMain"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c27b79e21 in std::__1::condition_variable::wait(std::__1::unique_lock&, Poco::Event::wait()::$_0) (this=0x7f940b006a40, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c27b79c9b in Poco::Event::wait (this=0x7f940b006a40) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Event.cpp:79 +#5 0x0000558c27bfbbb0 in Poco::PooledThread::run (this=0x7f940b006900) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/ThreadPool.cpp:206 +#6 0x0000558c27bf81ea in Poco::(anonymous namespace)::RunnableHolder::run (this=0x7f94936ee5e0) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread.cpp:43 +#7 0x0000558c27bf72ea in Poco::ThreadImpl::runnableEntry (pThread=0x7f940b006938) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread_STD.cpp:139 +#8 0x0000558c27bf9760 in std::__1::__invoke[abi:ue170006](void* (*&&)(void*), Poco::ThreadImpl*&&) (__f=@0x7f95702b77e8: 0x558c27bf7280 , __args=@0x7f95702b77f0: 0x7f940b006938) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#9 0x0000558c27bf972e in std::__1::__thread_execute[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*, 2ul>(std::__1::tuple >, void* (*)(void*), Poco::ThreadImpl*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#10 0x0000558c27bf9512 in std::__1::__thread_proxy[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*> >(void*) (__vp=0x7f95702b77e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#11 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#12 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 747 (Thread 0x7f927a83e640 (LWP 2740410) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6389668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6389600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6389600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6389600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=399) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ad30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86ad40: 0x7f956f09f800, __args=@0x7f940b86ad48: 399) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ad30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86ad48: 399, __args=@0x7f940b86ad48: 399) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86ad08, args=@0x7f940b86ad48: 399, args=@0x7f940b86ad48: 399) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86ad48: 399, __args=@0x7f940b86ad48: 399) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86ad00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 746 (Thread 0x7f927b23f640 (LWP 2740409) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6389368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6389300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6389300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6389300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=398) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ace0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86acf0: 0x7f956f09f800, __args=@0x7f940b86acf8: 398) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ace0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86acf8: 398, __args=@0x7f940b86acf8: 398) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86acb8, args=@0x7f940b86acf8: 398, args=@0x7f940b86acf8: 398) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86acf8: 398, __args=@0x7f940b86acf8: 398) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86acb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 745 (Thread 0x7f927bc40640 (LWP 2740408) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6389068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6389000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6389000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6389000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=397) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ac90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86aca0: 0x7f956f09f800, __args=@0x7f940b86aca8: 397) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ac90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86aca8: 397, __args=@0x7f940b86aca8: 397) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86ac68, args=@0x7f940b86aca8: 397, args=@0x7f940b86aca8: 397) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86aca8: 397, __args=@0x7f940b86aca8: 397) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86ac60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 744 (Thread 0x7f927c641640 (LWP 2740407) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6308d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6308d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6308d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6308d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=396) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ac40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86ac50: 0x7f956f09f800, __args=@0x7f940b86ac58: 396) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ac40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86ac58: 396, __args=@0x7f940b86ac58: 396) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86ac18, args=@0x7f940b86ac58: 396, args=@0x7f940b86ac58: 396) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86ac58: 396, __args=@0x7f940b86ac58: 396) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86ac10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 743 (Thread 0x7f927d042640 (LWP 2740406) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6308a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6308a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6308a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6308a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=395) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86abf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86ac00: 0x7f956f09f800, __args=@0x7f940b86ac08: 395) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86abf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86ac08: 395, __args=@0x7f940b86ac08: 395) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86abc8, args=@0x7f940b86ac08: 395, args=@0x7f940b86ac08: 395) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86ac08: 395, __args=@0x7f940b86ac08: 395) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86abc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 742 (Thread 0x7f927da43640 (LWP 2740405) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6308768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6308700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6308700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6308700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=394) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aba0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86abb0: 0x7f956f09f800, __args=@0x7f940b86abb8: 394) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aba0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86abb8: 394, __args=@0x7f940b86abb8: 394) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86ab78, args=@0x7f940b86abb8: 394, args=@0x7f940b86abb8: 394) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86abb8: 394, __args=@0x7f940b86abb8: 394) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86ab70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 741 (Thread 0x7f927e444640 (LWP 2740404) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6308468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6308400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6308400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6308400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=393) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ab50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86ab60: 0x7f956f09f800, __args=@0x7f940b86ab68: 393) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ab50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86ab68: 393, __args=@0x7f940b86ab68: 393) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86ab28, args=@0x7f940b86ab68: 393, args=@0x7f940b86ab68: 393) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86ab68: 393, __args=@0x7f940b86ab68: 393) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86ab20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 740 (Thread 0x7f927ee45640 (LWP 2740403) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6308168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6308100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6308100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6308100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=392) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ab00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86ab10: 0x7f956f09f800, __args=@0x7f940b86ab18: 392) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ab00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86ab18: 392, __args=@0x7f940b86ab18: 392) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86aad8, args=@0x7f940b86ab18: 392, args=@0x7f940b86ab18: 392) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86ab18: 392, __args=@0x7f940b86ab18: 392) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86aad0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 739 (Thread 0x7f927f846640 (LWP 2740402) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6307e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6307e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6307e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6307e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=391) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aab0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86aac0: 0x7f956f09f800, __args=@0x7f940b86aac8: 391) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aab0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86aac8: 391, __args=@0x7f940b86aac8: 391) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86aa88, args=@0x7f940b86aac8: 391, args=@0x7f940b86aac8: 391) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86aac8: 391, __args=@0x7f940b86aac8: 391) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86aa80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 738 (Thread 0x7f9280247640 (LWP 2740401) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6307b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6307b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6307b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6307b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=390) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aa60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86aa70: 0x7f956f09f800, __args=@0x7f940b86aa78: 390) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aa60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86aa78: 390, __args=@0x7f940b86aa78: 390) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86aa38, args=@0x7f940b86aa78: 390, args=@0x7f940b86aa78: 390) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86aa78: 390, __args=@0x7f940b86aa78: 390) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86aa30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 737 (Thread 0x7f9281848640 (LWP 2740400) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6307868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6307800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6307800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6307800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=389) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aa10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86aa20: 0x7f956f09f800, __args=@0x7f940b86aa28: 389) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aa10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86aa28: 389, __args=@0x7f940b86aa28: 389) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a9e8, args=@0x7f940b86aa28: 389, args=@0x7f940b86aa28: 389) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86aa28: 389, __args=@0x7f940b86aa28: 389) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a9e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 736 (Thread 0x7f9282249640 (LWP 2740399) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6307568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6307500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6307500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6307500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=388) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a9c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a9d0: 0x7f956f09f800, __args=@0x7f940b86a9d8: 388) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a9c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a9d8: 388, __args=@0x7f940b86a9d8: 388) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a998, args=@0x7f940b86a9d8: 388, args=@0x7f940b86a9d8: 388) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a9d8: 388, __args=@0x7f940b86a9d8: 388) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a990) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 735 (Thread 0x7f9282c4a640 (LWP 2740398) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6307268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6307200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6307200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6307200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=387) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a970: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a980: 0x7f956f09f800, __args=@0x7f940b86a988: 387) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a970: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a988: 387, __args=@0x7f940b86a988: 387) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a948, args=@0x7f940b86a988: 387, args=@0x7f940b86a988: 387) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a988: 387, __args=@0x7f940b86a988: 387) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 734 (Thread 0x7f928364b640 (LWP 2740397) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=386) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a920: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a930: 0x7f956f09f800, __args=@0x7f940b86a938: 386) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a920: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a938: 386, __args=@0x7f940b86a938: 386) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a8f8, args=@0x7f940b86a938: 386, args=@0x7f940b86a938: 386) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a938: 386, __args=@0x7f940b86a938: 386) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a8f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 733 (Thread 0x7f928404c640 (LWP 2740396) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=385) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a8d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a8e0: 0x7f956f09f800, __args=@0x7f940b86a8e8: 385) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a8d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a8e8: 385, __args=@0x7f940b86a8e8: 385) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a8a8, args=@0x7f940b86a8e8: 385, args=@0x7f940b86a8e8: 385) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a8e8: 385, __args=@0x7f940b86a8e8: 385) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a8a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 732 (Thread 0x7f9284a4d640 (LWP 2740395) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=384) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a880: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a890: 0x7f956f09f800, __args=@0x7f940b86a898: 384) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a880: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a898: 384, __args=@0x7f940b86a898: 384) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a858, args=@0x7f940b86a898: 384, args=@0x7f940b86a898: 384) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a898: 384, __args=@0x7f940b86a898: 384) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a850) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 731 (Thread 0x7f928544e640 (LWP 2740394) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=383) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a830: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a840: 0x7f956f09f800, __args=@0x7f940b86a848: 383) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a830: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a848: 383, __args=@0x7f940b86a848: 383) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a808, args=@0x7f940b86a848: 383, args=@0x7f940b86a848: 383) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a848: 383, __args=@0x7f940b86a848: 383) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a800) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 730 (Thread 0x7f9285e4f640 (LWP 2740393) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=382) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a7e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a7f0: 0x7f956f09f800, __args=@0x7f940b86a7f8: 382) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a7e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a7f8: 382, __args=@0x7f940b86a7f8: 382) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a7b8, args=@0x7f940b86a7f8: 382, args=@0x7f940b86a7f8: 382) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a7f8: 382, __args=@0x7f940b86a7f8: 382) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a7b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 729 (Thread 0x7f9286850640 (LWP 2740392) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=381) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a790: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a7a0: 0x7f956f09f800, __args=@0x7f940b86a7a8: 381) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a790: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a7a8: 381, __args=@0x7f940b86a7a8: 381) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a768, args=@0x7f940b86a7a8: 381, args=@0x7f940b86a7a8: 381) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a7a8: 381, __args=@0x7f940b86a7a8: 381) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a760) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 728 (Thread 0x7f9287251640 (LWP 2740391) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6285d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6285d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6285d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6285d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=380) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a740: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a750: 0x7f956f09f800, __args=@0x7f940b86a758: 380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a740: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a758: 380, __args=@0x7f940b86a758: 380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a718, args=@0x7f940b86a758: 380, args=@0x7f940b86a758: 380) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a758: 380, __args=@0x7f940b86a758: 380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a710) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 727 (Thread 0x7f9287c52640 (LWP 2740390) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6285a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6285a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6285a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6285a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=379) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a6f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a700: 0x7f956f09f800, __args=@0x7f940b86a708: 379) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a6f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a708: 379, __args=@0x7f940b86a708: 379) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a6c8, args=@0x7f940b86a708: 379, args=@0x7f940b86a708: 379) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a708: 379, __args=@0x7f940b86a708: 379) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a6c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 726 (Thread 0x7f9288653640 (LWP 2740389) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6285768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6285700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6285700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6285700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=378) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a6a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a6b0: 0x7f956f09f800, __args=@0x7f940b86a6b8: 378) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a6a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a6b8: 378, __args=@0x7f940b86a6b8: 378) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a678, args=@0x7f940b86a6b8: 378, args=@0x7f940b86a6b8: 378) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a6b8: 378, __args=@0x7f940b86a6b8: 378) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a670) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 725 (Thread 0x7f9289054640 (LWP 2740388) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6285468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6285400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6285400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6285400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=377) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a650: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a660: 0x7f956f09f800, __args=@0x7f940b86a668: 377) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a650: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a668: 377, __args=@0x7f940b86a668: 377) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a628, args=@0x7f940b86a668: 377, args=@0x7f940b86a668: 377) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a668: 377, __args=@0x7f940b86a668: 377) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a620) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 724 (Thread 0x7f9289a55640 (LWP 2740387) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6285168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6285100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6285100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6285100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=376) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a600: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a610: 0x7f956f09f800, __args=@0x7f940b86a618: 376) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a600: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a618: 376, __args=@0x7f940b86a618: 376) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a5d8, args=@0x7f940b86a618: 376, args=@0x7f940b86a618: 376) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a618: 376, __args=@0x7f940b86a618: 376) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a5d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 723 (Thread 0x7f928a456640 (LWP 2740386) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6284e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6284e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6284e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6284e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=375) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a5b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a5c0: 0x7f956f09f800, __args=@0x7f940b86a5c8: 375) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a5b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a5c8: 375, __args=@0x7f940b86a5c8: 375) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a588, args=@0x7f940b86a5c8: 375, args=@0x7f940b86a5c8: 375) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a5c8: 375, __args=@0x7f940b86a5c8: 375) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 722 (Thread 0x7f928ae57640 (LWP 2740385) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6284b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6284b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6284b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6284b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=374) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a560: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a570: 0x7f956f09f800, __args=@0x7f940b86a578: 374) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a560: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a578: 374, __args=@0x7f940b86a578: 374) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a538, args=@0x7f940b86a578: 374, args=@0x7f940b86a578: 374) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a578: 374, __args=@0x7f940b86a578: 374) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a530) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 721 (Thread 0x7f928b858640 (LWP 2740384) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6284868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6284800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6284800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6284800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=373) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a510: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a520: 0x7f956f09f800, __args=@0x7f940b86a528: 373) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a510: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a528: 373, __args=@0x7f940b86a528: 373) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a4e8, args=@0x7f940b86a528: 373, args=@0x7f940b86a528: 373) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a528: 373, __args=@0x7f940b86a528: 373) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a4e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 720 (Thread 0x7f928c259640 (LWP 2740383) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6284568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6284500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6284500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6284500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=372) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a4c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a4d0: 0x7f956f09f800, __args=@0x7f940b86a4d8: 372) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a4c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a4d8: 372, __args=@0x7f940b86a4d8: 372) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a498, args=@0x7f940b86a4d8: 372, args=@0x7f940b86a4d8: 372) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a4d8: 372, __args=@0x7f940b86a4d8: 372) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a490) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 719 (Thread 0x7f928cc5a640 (LWP 2740382) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6284268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6284200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6284200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6284200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=371) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a470: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a480: 0x7f956f09f800, __args=@0x7f940b86a488: 371) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a470: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a488: 371, __args=@0x7f940b86a488: 371) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a448, args=@0x7f940b86a488: 371, args=@0x7f940b86a488: 371) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a488: 371, __args=@0x7f940b86a488: 371) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 718 (Thread 0x7f928d65b640 (LWP 2740381) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=370) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a420: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a430: 0x7f956f09f800, __args=@0x7f940b86a438: 370) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a420: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a438: 370, __args=@0x7f940b86a438: 370) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a3f8, args=@0x7f940b86a438: 370, args=@0x7f940b86a438: 370) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a438: 370, __args=@0x7f940b86a438: 370) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a3f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 717 (Thread 0x7f928e05c640 (LWP 2740380) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=369) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a3d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a3e0: 0x7f956f09f800, __args=@0x7f940b86a3e8: 369) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a3d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a3e8: 369, __args=@0x7f940b86a3e8: 369) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a3a8, args=@0x7f940b86a3e8: 369, args=@0x7f940b86a3e8: 369) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a3e8: 369, __args=@0x7f940b86a3e8: 369) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a3a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 716 (Thread 0x7f928ea5d640 (LWP 2740379) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=368) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a380: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a390: 0x7f956f09f800, __args=@0x7f940b86a398: 368) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a380: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a398: 368, __args=@0x7f940b86a398: 368) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a358, args=@0x7f940b86a398: 368, args=@0x7f940b86a398: 368) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a398: 368, __args=@0x7f940b86a398: 368) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 715 (Thread 0x7f928f45e640 (LWP 2740378) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=367) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a330: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a340: 0x7f956f09f800, __args=@0x7f940b86a348: 367) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a330: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a348: 367, __args=@0x7f940b86a348: 367) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a308, args=@0x7f940b86a348: 367, args=@0x7f940b86a348: 367) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a348: 367, __args=@0x7f940b86a348: 367) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 714 (Thread 0x7f928fe5f640 (LWP 2740377) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=366) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a2e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a2f0: 0x7f956f09f800, __args=@0x7f940b86a2f8: 366) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a2e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a2f8: 366, __args=@0x7f940b86a2f8: 366) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a2b8, args=@0x7f940b86a2f8: 366, args=@0x7f940b86a2f8: 366) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a2f8: 366, __args=@0x7f940b86a2f8: 366) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a2b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 713 (Thread 0x7f9290860640 (LWP 2740376) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=365) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a290: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a2a0: 0x7f956f09f800, __args=@0x7f940b86a2a8: 365) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a290: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a2a8: 365, __args=@0x7f940b86a2a8: 365) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a268, args=@0x7f940b86a2a8: 365, args=@0x7f940b86a2a8: 365) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a2a8: 365, __args=@0x7f940b86a2a8: 365) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 712 (Thread 0x7f9291261640 (LWP 2740373) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61ffd68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61ffd00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61ffd00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61ffd00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=364) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a240: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a250: 0x7f956f09f800, __args=@0x7f940b86a258: 364) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a240: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a258: 364, __args=@0x7f940b86a258: 364) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a218, args=@0x7f940b86a258: 364, args=@0x7f940b86a258: 364) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a258: 364, __args=@0x7f940b86a258: 364) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a210) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 711 (Thread 0x7f9291c62640 (LWP 2740372) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61ffa68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61ffa00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61ffa00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61ffa00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=363) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a1f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a200: 0x7f956f09f800, __args=@0x7f940b86a208: 363) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a1f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a208: 363, __args=@0x7f940b86a208: 363) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a1c8, args=@0x7f940b86a208: 363, args=@0x7f940b86a208: 363) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a208: 363, __args=@0x7f940b86a208: 363) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a1c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 710 (Thread 0x7f9292663640 (LWP 2740371) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61ff768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61ff700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61ff700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61ff700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=362) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a1a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a1b0: 0x7f956f09f800, __args=@0x7f940b86a1b8: 362) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a1a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a1b8: 362, __args=@0x7f940b86a1b8: 362) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a178, args=@0x7f940b86a1b8: 362, args=@0x7f940b86a1b8: 362) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a1b8: 362, __args=@0x7f940b86a1b8: 362) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a170) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 709 (Thread 0x7f9293064640 (LWP 2740370) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61ff468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61ff400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61ff400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61ff400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=361) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a150: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a160: 0x7f956f09f800, __args=@0x7f940b86a168: 361) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a150: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a168: 361, __args=@0x7f940b86a168: 361) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a128, args=@0x7f940b86a168: 361, args=@0x7f940b86a168: 361) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a168: 361, __args=@0x7f940b86a168: 361) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a120) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 708 (Thread 0x7f9293a65640 (LWP 2740369) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61ff168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61ff100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61ff100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61ff100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=360) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a100: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a110: 0x7f956f09f800, __args=@0x7f940b86a118: 360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a100: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a118: 360, __args=@0x7f940b86a118: 360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a0d8, args=@0x7f940b86a118: 360, args=@0x7f940b86a118: 360) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a118: 360, __args=@0x7f940b86a118: 360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a0d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 707 (Thread 0x7f9294466640 (LWP 2740368) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fee68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fee00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fee00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fee00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=359) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a0b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a0c0: 0x7f956f09f800, __args=@0x7f940b86a0c8: 359) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a0b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a0c8: 359, __args=@0x7f940b86a0c8: 359) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a088, args=@0x7f940b86a0c8: 359, args=@0x7f940b86a0c8: 359) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a0c8: 359, __args=@0x7f940b86a0c8: 359) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 706 (Thread 0x7f9294e67640 (LWP 2740367) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61feb68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61feb00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61feb00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61feb00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=358) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a060: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a070: 0x7f956f09f800, __args=@0x7f940b86a078: 358) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a060: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a078: 358, __args=@0x7f940b86a078: 358) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a038, args=@0x7f940b86a078: 358, args=@0x7f940b86a078: 358) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a078: 358, __args=@0x7f940b86a078: 358) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a030) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 705 (Thread 0x7f9295868640 (LWP 2740366) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fe868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fe800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fe800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fe800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=357) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a010: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a020: 0x7f956f09f800, __args=@0x7f940b86a028: 357) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a010: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a028: 357, __args=@0x7f940b86a028: 357) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869fe8, args=@0x7f940b86a028: 357, args=@0x7f940b86a028: 357) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a028: 357, __args=@0x7f940b86a028: 357) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869fe0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 704 (Thread 0x7f9296269640 (LWP 2740365) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fe568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fe500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fe500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fe500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=356) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869fc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869fd0: 0x7f956f09f800, __args=@0x7f940b869fd8: 356) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869fc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869fd8: 356, __args=@0x7f940b869fd8: 356) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869f98, args=@0x7f940b869fd8: 356, args=@0x7f940b869fd8: 356) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869fd8: 356, __args=@0x7f940b869fd8: 356) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869f90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 703 (Thread 0x7f9296c6a640 (LWP 2740364) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fe268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fe200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fe200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fe200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=355) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869f70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869f80: 0x7f956f09f800, __args=@0x7f940b869f88: 355) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869f70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869f88: 355, __args=@0x7f940b869f88: 355) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869f48, args=@0x7f940b869f88: 355, args=@0x7f940b869f88: 355) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869f88: 355, __args=@0x7f940b869f88: 355) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869f40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 702 (Thread 0x7f929766b640 (LWP 2740363) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fdf68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fdf00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fdf00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fdf00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=354) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869f20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869f30: 0x7f956f09f800, __args=@0x7f940b869f38: 354) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869f20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869f38: 354, __args=@0x7f940b869f38: 354) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869ef8, args=@0x7f940b869f38: 354, args=@0x7f940b869f38: 354) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869f38: 354, __args=@0x7f940b869f38: 354) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869ef0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 701 (Thread 0x7f929806c640 (LWP 2740362) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fdc68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fdc00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fdc00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fdc00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=353) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ed0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869ee0: 0x7f956f09f800, __args=@0x7f940b869ee8: 353) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ed0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869ee8: 353, __args=@0x7f940b869ee8: 353) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869ea8, args=@0x7f940b869ee8: 353, args=@0x7f940b869ee8: 353) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869ee8: 353, __args=@0x7f940b869ee8: 353) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869ea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 700 (Thread 0x7f9298a6d640 (LWP 2740361) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fd968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fd900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fd900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fd900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=352) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869e80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869e90: 0x7f956f09f800, __args=@0x7f940b869e98: 352) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869e80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869e98: 352, __args=@0x7f940b869e98: 352) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869e58, args=@0x7f940b869e98: 352, args=@0x7f940b869e98: 352) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869e98: 352, __args=@0x7f940b869e98: 352) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869e50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 699 (Thread 0x7f929946e640 (LWP 2740360) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fd668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fd600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fd600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fd600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=351) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869e30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869e40: 0x7f956f09f800, __args=@0x7f940b869e48: 351) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869e30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869e48: 351, __args=@0x7f940b869e48: 351) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869e08, args=@0x7f940b869e48: 351, args=@0x7f940b869e48: 351) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869e48: 351, __args=@0x7f940b869e48: 351) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869e00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 698 (Thread 0x7f9299e6f640 (LWP 2740359) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fd368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fd300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fd300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fd300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=350) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869de0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869df0: 0x7f956f09f800, __args=@0x7f940b869df8: 350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869de0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869df8: 350, __args=@0x7f940b869df8: 350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869db8, args=@0x7f940b869df8: 350, args=@0x7f940b869df8: 350) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869df8: 350, __args=@0x7f940b869df8: 350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869db0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 697 (Thread 0x7f929a870640 (LWP 2740358) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fd068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fd000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fd000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fd000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=349) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869d90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869da0: 0x7f956f09f800, __args=@0x7f940b869da8: 349) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869d90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869da8: 349, __args=@0x7f940b869da8: 349) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869d68, args=@0x7f940b869da8: 349, args=@0x7f940b869da8: 349) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869da8: 349, __args=@0x7f940b869da8: 349) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869d60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 696 (Thread 0x7f929b271640 (LWP 2740357) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0bd68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0bd00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0bd00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0bd00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=348) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869d40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869d50: 0x7f956f09f800, __args=@0x7f940b869d58: 348) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869d40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869d58: 348, __args=@0x7f940b869d58: 348) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869d18, args=@0x7f940b869d58: 348, args=@0x7f940b869d58: 348) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869d58: 348, __args=@0x7f940b869d58: 348) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869d10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 695 (Thread 0x7f929bc72640 (LWP 2740356) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0ba68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0ba00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0ba00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0ba00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=347) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869cf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869d00: 0x7f956f09f800, __args=@0x7f940b869d08: 347) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869cf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869d08: 347, __args=@0x7f940b869d08: 347) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869cc8, args=@0x7f940b869d08: 347, args=@0x7f940b869d08: 347) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869d08: 347, __args=@0x7f940b869d08: 347) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 694 (Thread 0x7f929c673640 (LWP 2740355) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0b768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0b700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0b700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0b700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=346) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ca0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869cb0: 0x7f956f09f800, __args=@0x7f940b869cb8: 346) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ca0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869cb8: 346, __args=@0x7f940b869cb8: 346) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869c78, args=@0x7f940b869cb8: 346, args=@0x7f940b869cb8: 346) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869cb8: 346, __args=@0x7f940b869cb8: 346) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869c70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 693 (Thread 0x7f929d074640 (LWP 2740354) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0b468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0b400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0b400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0b400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=345) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869c50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869c60: 0x7f956f09f800, __args=@0x7f940b869c68: 345) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869c50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869c68: 345, __args=@0x7f940b869c68: 345) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869c28, args=@0x7f940b869c68: 345, args=@0x7f940b869c68: 345) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869c68: 345, __args=@0x7f940b869c68: 345) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869c20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 692 (Thread 0x7f929da75640 (LWP 2740353) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0b168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0b100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0b100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0b100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=344) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869c00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869c10: 0x7f956f09f800, __args=@0x7f940b869c18: 344) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869c00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869c18: 344, __args=@0x7f940b869c18: 344) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869bd8, args=@0x7f940b869c18: 344, args=@0x7f940b869c18: 344) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869c18: 344, __args=@0x7f940b869c18: 344) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869bd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 691 (Thread 0x7f929e476640 (LWP 2740352) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0ae68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0ae00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0ae00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0ae00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=343) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869bb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869bc0: 0x7f956f09f800, __args=@0x7f940b869bc8: 343) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869bb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869bc8: 343, __args=@0x7f940b869bc8: 343) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869b88, args=@0x7f940b869bc8: 343, args=@0x7f940b869bc8: 343) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869bc8: 343, __args=@0x7f940b869bc8: 343) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869b80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 690 (Thread 0x7f929ee77640 (LWP 2740351) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0ab68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0ab00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0ab00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0ab00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=342) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869b60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869b70: 0x7f956f09f800, __args=@0x7f940b869b78: 342) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869b60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869b78: 342, __args=@0x7f940b869b78: 342) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869b38, args=@0x7f940b869b78: 342, args=@0x7f940b869b78: 342) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869b78: 342, __args=@0x7f940b869b78: 342) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869b30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 689 (Thread 0x7f929f878640 (LWP 2740350) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0a868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0a800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0a800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0a800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=341) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869b10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869b20: 0x7f956f09f800, __args=@0x7f940b869b28: 341) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869b10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869b28: 341, __args=@0x7f940b869b28: 341) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869ae8, args=@0x7f940b869b28: 341, args=@0x7f940b869b28: 341) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869b28: 341, __args=@0x7f940b869b28: 341) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869ae0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 688 (Thread 0x7f92a0279640 (LWP 2740349) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0a568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0a500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0a500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0a500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=340) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ac0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869ad0: 0x7f956f09f800, __args=@0x7f940b869ad8: 340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ac0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869ad8: 340, __args=@0x7f940b869ad8: 340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869a98, args=@0x7f940b869ad8: 340, args=@0x7f940b869ad8: 340) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869ad8: 340, __args=@0x7f940b869ad8: 340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869a90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 687 (Thread 0x7f92a0c7a640 (LWP 2740348) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0a268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0a200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0a200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0a200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=339) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869a70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869a80: 0x7f956f09f800, __args=@0x7f940b869a88: 339) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869a70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869a88: 339, __args=@0x7f940b869a88: 339) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869a48, args=@0x7f940b869a88: 339, args=@0x7f940b869a88: 339) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869a88: 339, __args=@0x7f940b869a88: 339) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869a40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 686 (Thread 0x7f92a167b640 (LWP 2740347) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=338) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869a20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869a30: 0x7f956f09f800, __args=@0x7f940b869a38: 338) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869a20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869a38: 338, __args=@0x7f940b869a38: 338) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8699f8, args=@0x7f940b869a38: 338, args=@0x7f940b869a38: 338) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869a38: 338, __args=@0x7f940b869a38: 338) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8699f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 685 (Thread 0x7f92a207c640 (LWP 2740346) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=337) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8699d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8699e0: 0x7f956f09f800, __args=@0x7f940b8699e8: 337) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8699d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8699e8: 337, __args=@0x7f940b8699e8: 337) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8699a8, args=@0x7f940b8699e8: 337, args=@0x7f940b8699e8: 337) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8699e8: 337, __args=@0x7f940b8699e8: 337) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8699a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 684 (Thread 0x7f92a2a7d640 (LWP 2740345) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=336) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869980: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869990: 0x7f956f09f800, __args=@0x7f940b869998: 336) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869980: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869998: 336, __args=@0x7f940b869998: 336) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869958, args=@0x7f940b869998: 336, args=@0x7f940b869998: 336) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869998: 336, __args=@0x7f940b869998: 336) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869950) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 683 (Thread 0x7f92a347e640 (LWP 2740344) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=335) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869930: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869940: 0x7f956f09f800, __args=@0x7f940b869948: 335) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869930: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869948: 335, __args=@0x7f940b869948: 335) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869908, args=@0x7f940b869948: 335, args=@0x7f940b869948: 335) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869948: 335, __args=@0x7f940b869948: 335) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 682 (Thread 0x7f92a3e7f640 (LWP 2740343) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=334) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8698e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8698f0: 0x7f956f09f800, __args=@0x7f940b8698f8: 334) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8698e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8698f8: 334, __args=@0x7f940b8698f8: 334) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8698b8, args=@0x7f940b8698f8: 334, args=@0x7f940b8698f8: 334) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8698f8: 334, __args=@0x7f940b8698f8: 334) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8698b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 681 (Thread 0x7f92a4880640 (LWP 2740342) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=333) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869890: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8698a0: 0x7f956f09f800, __args=@0x7f940b8698a8: 333) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869890: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8698a8: 333, __args=@0x7f940b8698a8: 333) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869868, args=@0x7f940b8698a8: 333, args=@0x7f940b8698a8: 333) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8698a8: 333, __args=@0x7f940b8698a8: 333) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869860) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 680 (Thread 0x7f92a5281640 (LWP 2740341) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba08d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba08d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba08d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba08d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=332) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869840: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869850: 0x7f956f09f800, __args=@0x7f940b869858: 332) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869840: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869858: 332, __args=@0x7f940b869858: 332) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869818, args=@0x7f940b869858: 332, args=@0x7f940b869858: 332) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869858: 332, __args=@0x7f940b869858: 332) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869810) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 679 (Thread 0x7f92a5c82640 (LWP 2740340) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba08a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba08a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba08a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba08a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=331) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8697f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869800: 0x7f956f09f800, __args=@0x7f940b869808: 331) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8697f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869808: 331, __args=@0x7f940b869808: 331) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8697c8, args=@0x7f940b869808: 331, args=@0x7f940b869808: 331) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869808: 331, __args=@0x7f940b869808: 331) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8697c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 678 (Thread 0x7f92a6683640 (LWP 2740339) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba08768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba08700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba08700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba08700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=330) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8697a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8697b0: 0x7f956f09f800, __args=@0x7f940b8697b8: 330) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8697a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8697b8: 330, __args=@0x7f940b8697b8: 330) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869778, args=@0x7f940b8697b8: 330, args=@0x7f940b8697b8: 330) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8697b8: 330, __args=@0x7f940b8697b8: 330) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869770) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 677 (Thread 0x7f92a7084640 (LWP 2740338) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba08468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba08400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba08400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba08400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=329) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869750: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869760: 0x7f956f09f800, __args=@0x7f940b869768: 329) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869750: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869768: 329, __args=@0x7f940b869768: 329) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869728, args=@0x7f940b869768: 329, args=@0x7f940b869768: 329) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869768: 329, __args=@0x7f940b869768: 329) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 676 (Thread 0x7f92a7a85640 (LWP 2740337) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba08168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba08100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba08100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba08100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=328) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869700: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869710: 0x7f956f09f800, __args=@0x7f940b869718: 328) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869700: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869718: 328, __args=@0x7f940b869718: 328) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8696d8, args=@0x7f940b869718: 328, args=@0x7f940b869718: 328) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869718: 328, __args=@0x7f940b869718: 328) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8696d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 675 (Thread 0x7f92a8486640 (LWP 2740336) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba07e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba07e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba07e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba07e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=327) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8696b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8696c0: 0x7f956f09f800, __args=@0x7f940b8696c8: 327) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8696b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8696c8: 327, __args=@0x7f940b8696c8: 327) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869688, args=@0x7f940b8696c8: 327, args=@0x7f940b8696c8: 327) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8696c8: 327, __args=@0x7f940b8696c8: 327) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 674 (Thread 0x7f92a8e87640 (LWP 2740335) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba07b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba07b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba07b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba07b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=326) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869660: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869670: 0x7f956f09f800, __args=@0x7f940b869678: 326) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869660: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869678: 326, __args=@0x7f940b869678: 326) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869638, args=@0x7f940b869678: 326, args=@0x7f940b869678: 326) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869678: 326, __args=@0x7f940b869678: 326) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869630) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 673 (Thread 0x7f92a9888640 (LWP 2740334) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba07868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba07800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba07800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba07800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=325) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869610: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869620: 0x7f956f09f800, __args=@0x7f940b869628: 325) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869610: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869628: 325, __args=@0x7f940b869628: 325) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8695e8, args=@0x7f940b869628: 325, args=@0x7f940b869628: 325) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869628: 325, __args=@0x7f940b869628: 325) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8695e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 672 (Thread 0x7f92aa289640 (LWP 2740333) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba07568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba07500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba07500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba07500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=324) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8695c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8695d0: 0x7f956f09f800, __args=@0x7f940b8695d8: 324) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8695c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8695d8: 324, __args=@0x7f940b8695d8: 324) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869598, args=@0x7f940b8695d8: 324, args=@0x7f940b8695d8: 324) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8695d8: 324, __args=@0x7f940b8695d8: 324) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869590) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 671 (Thread 0x7f92aac8a640 (LWP 2740332) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba07268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba07200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba07200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba07200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=323) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869570: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869580: 0x7f956f09f800, __args=@0x7f940b869588: 323) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869570: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869588: 323, __args=@0x7f940b869588: 323) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869548, args=@0x7f940b869588: 323, args=@0x7f940b869588: 323) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869588: 323, __args=@0x7f940b869588: 323) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 670 (Thread 0x7f92ab68b640 (LWP 2740331) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=322) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869520: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869530: 0x7f956f09f800, __args=@0x7f940b869538: 322) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869520: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869538: 322, __args=@0x7f940b869538: 322) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8694f8, args=@0x7f940b869538: 322, args=@0x7f940b869538: 322) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869538: 322, __args=@0x7f940b869538: 322) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8694f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 669 (Thread 0x7f92ac08c640 (LWP 2740330) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=321) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8694d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8694e0: 0x7f956f09f800, __args=@0x7f940b8694e8: 321) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8694d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8694e8: 321, __args=@0x7f940b8694e8: 321) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8694a8, args=@0x7f940b8694e8: 321, args=@0x7f940b8694e8: 321) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8694e8: 321, __args=@0x7f940b8694e8: 321) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8694a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 668 (Thread 0x7f92aca8d640 (LWP 2740329) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=320) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869480: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869490: 0x7f956f09f800, __args=@0x7f940b869498: 320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869480: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869498: 320, __args=@0x7f940b869498: 320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869458, args=@0x7f940b869498: 320, args=@0x7f940b869498: 320) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869498: 320, __args=@0x7f940b869498: 320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869450) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 667 (Thread 0x7f92ad48e640 (LWP 2740328) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=319) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869430: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869440: 0x7f956f09f800, __args=@0x7f940b869448: 319) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869430: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869448: 319, __args=@0x7f940b869448: 319) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869408, args=@0x7f940b869448: 319, args=@0x7f940b869448: 319) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869448: 319, __args=@0x7f940b869448: 319) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 666 (Thread 0x7f92ade8f640 (LWP 2740327) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=318) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8693e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8693f0: 0x7f956f09f800, __args=@0x7f940b8693f8: 318) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8693e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8693f8: 318, __args=@0x7f940b8693f8: 318) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8693b8, args=@0x7f940b8693f8: 318, args=@0x7f940b8693f8: 318) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8693f8: 318, __args=@0x7f940b8693f8: 318) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8693b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 665 (Thread 0x7f92ae890640 (LWP 2740326) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=317) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869390: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8693a0: 0x7f956f09f800, __args=@0x7f940b8693a8: 317) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869390: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8693a8: 317, __args=@0x7f940b8693a8: 317) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869368, args=@0x7f940b8693a8: 317, args=@0x7f940b8693a8: 317) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8693a8: 317, __args=@0x7f940b8693a8: 317) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 664 (Thread 0x7f92af291640 (LWP 2740325) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e5d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e5d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e5d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e5d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=316) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869340: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869350: 0x7f956f09f800, __args=@0x7f940b869358: 316) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869340: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869358: 316, __args=@0x7f940b869358: 316) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869318, args=@0x7f940b869358: 316, args=@0x7f940b869358: 316) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869358: 316, __args=@0x7f940b869358: 316) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869310) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 663 (Thread 0x7f92afc92640 (LWP 2740324) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e5a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e5a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e5a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e5a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=315) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8692f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869300: 0x7f956f09f800, __args=@0x7f940b869308: 315) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8692f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869308: 315, __args=@0x7f940b869308: 315) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8692c8, args=@0x7f940b869308: 315, args=@0x7f940b869308: 315) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869308: 315, __args=@0x7f940b869308: 315) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8692c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 662 (Thread 0x7f92b0693640 (LWP 2740323) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e5768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e5700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e5700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e5700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=314) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8692a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8692b0: 0x7f956f09f800, __args=@0x7f940b8692b8: 314) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8692a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8692b8: 314, __args=@0x7f940b8692b8: 314) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869278, args=@0x7f940b8692b8: 314, args=@0x7f940b8692b8: 314) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8692b8: 314, __args=@0x7f940b8692b8: 314) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869270) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 661 (Thread 0x7f92b1094640 (LWP 2740322) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e5468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e5400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e5400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e5400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=313) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869250: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869260: 0x7f956f09f800, __args=@0x7f940b869268: 313) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869250: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869268: 313, __args=@0x7f940b869268: 313) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869228, args=@0x7f940b869268: 313, args=@0x7f940b869268: 313) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869268: 313, __args=@0x7f940b869268: 313) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 660 (Thread 0x7f92b1a95640 (LWP 2740321) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e5168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e5100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e5100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e5100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=312) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869200: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869210: 0x7f956f09f800, __args=@0x7f940b869218: 312) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869200: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869218: 312, __args=@0x7f940b869218: 312) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8691d8, args=@0x7f940b869218: 312, args=@0x7f940b869218: 312) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869218: 312, __args=@0x7f940b869218: 312) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8691d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 659 (Thread 0x7f92b2496640 (LWP 2740320) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e4e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e4e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e4e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e4e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=311) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8691b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8691c0: 0x7f956f09f800, __args=@0x7f940b8691c8: 311) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8691b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8691c8: 311, __args=@0x7f940b8691c8: 311) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869188, args=@0x7f940b8691c8: 311, args=@0x7f940b8691c8: 311) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8691c8: 311, __args=@0x7f940b8691c8: 311) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 658 (Thread 0x7f92b2e97640 (LWP 2740319) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e4b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e4b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e4b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e4b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=310) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869160: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869170: 0x7f956f09f800, __args=@0x7f940b869178: 310) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869160: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869178: 310, __args=@0x7f940b869178: 310) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869138, args=@0x7f940b869178: 310, args=@0x7f940b869178: 310) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869178: 310, __args=@0x7f940b869178: 310) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869130) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 657 (Thread 0x7f92b3898640 (LWP 2740318) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e4868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e4800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e4800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e4800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=309) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869110: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869120: 0x7f956f09f800, __args=@0x7f940b869128: 309) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869110: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869128: 309, __args=@0x7f940b869128: 309) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8690e8, args=@0x7f940b869128: 309, args=@0x7f940b869128: 309) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869128: 309, __args=@0x7f940b869128: 309) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8690e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 656 (Thread 0x7f92b4299640 (LWP 2740317) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e4568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e4500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e4500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e4500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=308) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8690c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8690d0: 0x7f956f09f800, __args=@0x7f940b8690d8: 308) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8690c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8690d8: 308, __args=@0x7f940b8690d8: 308) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869098, args=@0x7f940b8690d8: 308, args=@0x7f940b8690d8: 308) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8690d8: 308, __args=@0x7f940b8690d8: 308) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869090) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 655 (Thread 0x7f92b4c9a640 (LWP 2740316) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e4268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e4200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e4200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e4200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=307) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869070: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869080: 0x7f956f09f800, __args=@0x7f940b869088: 307) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869070: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869088: 307, __args=@0x7f940b869088: 307) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869048, args=@0x7f940b869088: 307, args=@0x7f940b869088: 307) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869088: 307, __args=@0x7f940b869088: 307) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 654 (Thread 0x7f92b569b640 (LWP 2740315) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=306) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869020: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869030: 0x7f956f09f800, __args=@0x7f940b869038: 306) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869020: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869038: 306, __args=@0x7f940b869038: 306) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868ff8, args=@0x7f940b869038: 306, args=@0x7f940b869038: 306) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869038: 306, __args=@0x7f940b869038: 306) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868ff0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 653 (Thread 0x7f92b609c640 (LWP 2740314) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=305) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868fd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868fe0: 0x7f956f09f800, __args=@0x7f940b868fe8: 305) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868fd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868fe8: 305, __args=@0x7f940b868fe8: 305) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868fa8, args=@0x7f940b868fe8: 305, args=@0x7f940b868fe8: 305) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868fe8: 305, __args=@0x7f940b868fe8: 305) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868fa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 652 (Thread 0x7f92b7a9d640 (LWP 2740313) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=304) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868f80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868f90: 0x7f956f09f800, __args=@0x7f940b868f98: 304) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868f80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868f98: 304, __args=@0x7f940b868f98: 304) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868f58, args=@0x7f940b868f98: 304, args=@0x7f940b868f98: 304) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868f98: 304, __args=@0x7f940b868f98: 304) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868f50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 651 (Thread 0x7f92b849e640 (LWP 2740312) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=303) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868f30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868f40: 0x7f956f09f800, __args=@0x7f940b868f48: 303) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868f30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868f48: 303, __args=@0x7f940b868f48: 303) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868f08, args=@0x7f940b868f48: 303, args=@0x7f940b868f48: 303) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868f48: 303, __args=@0x7f940b868f48: 303) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868f00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 650 (Thread 0x7f92b8e9f640 (LWP 2740311) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=302) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868ee0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868ef0: 0x7f956f09f800, __args=@0x7f940b868ef8: 302) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868ee0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868ef8: 302, __args=@0x7f940b868ef8: 302) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868eb8, args=@0x7f940b868ef8: 302, args=@0x7f940b868ef8: 302) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868ef8: 302, __args=@0x7f940b868ef8: 302) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868eb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 649 (Thread 0x7f92b98a0640 (LWP 2740310) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=301) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868e90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868ea0: 0x7f956f09f800, __args=@0x7f940b868ea8: 301) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868e90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868ea8: 301, __args=@0x7f940b868ea8: 301) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868e68, args=@0x7f940b868ea8: 301, args=@0x7f940b868ea8: 301) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868ea8: 301, __args=@0x7f940b868ea8: 301) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868e60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 648 (Thread 0x7f92ba2a1640 (LWP 2740309) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b962d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b962d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b962d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b962d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=300) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868e40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868e50: 0x7f956f09f800, __args=@0x7f940b868e58: 300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868e40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868e58: 300, __args=@0x7f940b868e58: 300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868e18, args=@0x7f940b868e58: 300, args=@0x7f940b868e58: 300) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868e58: 300, __args=@0x7f940b868e58: 300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868e10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 647 (Thread 0x7f92baca2640 (LWP 2740308) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b962a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b962a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b962a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b962a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=299) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868df0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868e00: 0x7f956f09f800, __args=@0x7f940b868e08: 299) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868df0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868e08: 299, __args=@0x7f940b868e08: 299) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868dc8, args=@0x7f940b868e08: 299, args=@0x7f940b868e08: 299) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868e08: 299, __args=@0x7f940b868e08: 299) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868dc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 646 (Thread 0x7f92bb6a3640 (LWP 2740307) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b962768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b962700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b962700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b962700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=298) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868da0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868db0: 0x7f956f09f800, __args=@0x7f940b868db8: 298) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868da0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868db8: 298, __args=@0x7f940b868db8: 298) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868d78, args=@0x7f940b868db8: 298, args=@0x7f940b868db8: 298) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868db8: 298, __args=@0x7f940b868db8: 298) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868d70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 645 (Thread 0x7f92bc0a4640 (LWP 2740306) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b962468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b962400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b962400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b962400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=297) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868d50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868d60: 0x7f956f09f800, __args=@0x7f940b868d68: 297) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868d50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868d68: 297, __args=@0x7f940b868d68: 297) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868d28, args=@0x7f940b868d68: 297, args=@0x7f940b868d68: 297) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868d68: 297, __args=@0x7f940b868d68: 297) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868d20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 644 (Thread 0x7f92bd4a5640 (LWP 2740305) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b962168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b962100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b962100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b962100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=296) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868d00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868d10: 0x7f956f09f800, __args=@0x7f940b868d18: 296) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868d00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868d18: 296, __args=@0x7f940b868d18: 296) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868cd8, args=@0x7f940b868d18: 296, args=@0x7f940b868d18: 296) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868d18: 296, __args=@0x7f940b868d18: 296) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868cd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 643 (Thread 0x7f92bdea6640 (LWP 2740304) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b961e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b961e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b961e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b961e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=295) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868cb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868cc0: 0x7f956f09f800, __args=@0x7f940b868cc8: 295) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868cb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868cc8: 295, __args=@0x7f940b868cc8: 295) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868c88, args=@0x7f940b868cc8: 295, args=@0x7f940b868cc8: 295) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868cc8: 295, __args=@0x7f940b868cc8: 295) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868c80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 642 (Thread 0x7f92be8a7640 (LWP 2740303) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b961b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b961b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b961b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b961b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=294) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868c60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868c70: 0x7f956f09f800, __args=@0x7f940b868c78: 294) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868c60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868c78: 294, __args=@0x7f940b868c78: 294) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868c38, args=@0x7f940b868c78: 294, args=@0x7f940b868c78: 294) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868c78: 294, __args=@0x7f940b868c78: 294) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868c30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 641 (Thread 0x7f92bf2a8640 (LWP 2740302) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b961868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b961800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b961800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b961800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=293) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868c10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868c20: 0x7f956f09f800, __args=@0x7f940b868c28: 293) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868c10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868c28: 293, __args=@0x7f940b868c28: 293) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868be8, args=@0x7f940b868c28: 293, args=@0x7f940b868c28: 293) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868c28: 293, __args=@0x7f940b868c28: 293) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868be0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 640 (Thread 0x7f92bfca9640 (LWP 2740301) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b961568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b961500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b961500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b961500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=292) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868bc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868bd0: 0x7f956f09f800, __args=@0x7f940b868bd8: 292) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868bc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868bd8: 292, __args=@0x7f940b868bd8: 292) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868b98, args=@0x7f940b868bd8: 292, args=@0x7f940b868bd8: 292) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868bd8: 292, __args=@0x7f940b868bd8: 292) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868b90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 639 (Thread 0x7f92c06aa640 (LWP 2740300) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b961268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b961200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b961200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b961200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=291) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868b70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868b80: 0x7f956f09f800, __args=@0x7f940b868b88: 291) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868b70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868b88: 291, __args=@0x7f940b868b88: 291) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868b48, args=@0x7f940b868b88: 291, args=@0x7f940b868b88: 291) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868b88: 291, __args=@0x7f940b868b88: 291) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 638 (Thread 0x7f92c10ab640 (LWP 2740299) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b960f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=290) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868b20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868b30: 0x7f956f09f800, __args=@0x7f940b868b38: 290) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868b20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868b38: 290, __args=@0x7f940b868b38: 290) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868af8, args=@0x7f940b868b38: 290, args=@0x7f940b868b38: 290) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868b38: 290, __args=@0x7f940b868b38: 290) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868af0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 637 (Thread 0x7f92c1aac640 (LWP 2740298) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b960c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=289) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868ad0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868ae0: 0x7f956f09f800, __args=@0x7f940b868ae8: 289) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868ad0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868ae8: 289, __args=@0x7f940b868ae8: 289) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868aa8, args=@0x7f940b868ae8: 289, args=@0x7f940b868ae8: 289) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868ae8: 289, __args=@0x7f940b868ae8: 289) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868aa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 636 (Thread 0x7f92c24ad640 (LWP 2740297) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b960900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=288) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868a80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868a90: 0x7f956f09f800, __args=@0x7f940b868a98: 288) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868a80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868a98: 288, __args=@0x7f940b868a98: 288) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868a58, args=@0x7f940b868a98: 288, args=@0x7f940b868a98: 288) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868a98: 288, __args=@0x7f940b868a98: 288) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868a50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 635 (Thread 0x7f92c2eae640 (LWP 2740296) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b960600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=287) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868a30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868a40: 0x7f956f09f800, __args=@0x7f940b868a48: 287) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868a30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868a48: 287, __args=@0x7f940b868a48: 287) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868a08, args=@0x7f940b868a48: 287, args=@0x7f940b868a48: 287) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868a48: 287, __args=@0x7f940b868a48: 287) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 634 (Thread 0x7f92c38af640 (LWP 2740295) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b960300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=286) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8689e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8689f0: 0x7f956f09f800, __args=@0x7f940b8689f8: 286) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8689e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8689f8: 286, __args=@0x7f940b8689f8: 286) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8689b8, args=@0x7f940b8689f8: 286, args=@0x7f940b8689f8: 286) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8689f8: 286, __args=@0x7f940b8689f8: 286) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8689b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 633 (Thread 0x7f92c42b0640 (LWP 2740294) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b960000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=285) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868990: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8689a0: 0x7f956f09f800, __args=@0x7f940b8689a8: 285) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868990: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8689a8: 285, __args=@0x7f940b8689a8: 285) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868968, args=@0x7f940b8689a8: 285, args=@0x7f940b8689a8: 285) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8689a8: 285, __args=@0x7f940b8689a8: 285) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868960) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 632 (Thread 0x7f92c4cb1640 (LWP 2740293) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dfd68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dfd00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dfd00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dfd00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=284) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868940: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868950: 0x7f956f09f800, __args=@0x7f940b868958: 284) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868940: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868958: 284, __args=@0x7f940b868958: 284) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868918, args=@0x7f940b868958: 284, args=@0x7f940b868958: 284) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868958: 284, __args=@0x7f940b868958: 284) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868910) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 631 (Thread 0x7f92c56b2640 (LWP 2740292) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dfa68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dfa00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dfa00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dfa00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=283) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8688f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868900: 0x7f956f09f800, __args=@0x7f940b868908: 283) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8688f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868908: 283, __args=@0x7f940b868908: 283) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8688c8, args=@0x7f940b868908: 283, args=@0x7f940b868908: 283) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868908: 283, __args=@0x7f940b868908: 283) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8688c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 630 (Thread 0x7f92c60b3640 (LWP 2740291) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8df768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8df700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8df700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8df700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=282) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8688a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8688b0: 0x7f956f09f800, __args=@0x7f940b8688b8: 282) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8688a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8688b8: 282, __args=@0x7f940b8688b8: 282) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868878, args=@0x7f940b8688b8: 282, args=@0x7f940b8688b8: 282) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8688b8: 282, __args=@0x7f940b8688b8: 282) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868870) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 629 (Thread 0x7f92c6ab4640 (LWP 2740290) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8df468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8df400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8df400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8df400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=281) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868850: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868860: 0x7f956f09f800, __args=@0x7f940b868868: 281) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868850: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868868: 281, __args=@0x7f940b868868: 281) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868828, args=@0x7f940b868868: 281, args=@0x7f940b868868: 281) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868868: 281, __args=@0x7f940b868868: 281) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868820) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 628 (Thread 0x7f92c74b5640 (LWP 2740289) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8df168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8df100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8df100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8df100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=280) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868800: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868810: 0x7f956f09f800, __args=@0x7f940b868818: 280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868800: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868818: 280, __args=@0x7f940b868818: 280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8687d8, args=@0x7f940b868818: 280, args=@0x7f940b868818: 280) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868818: 280, __args=@0x7f940b868818: 280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8687d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 627 (Thread 0x7f92c7eb6640 (LWP 2740288) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dee68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dee00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dee00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dee00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=279) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8687b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8687c0: 0x7f956f09f800, __args=@0x7f940b8687c8: 279) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8687b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8687c8: 279, __args=@0x7f940b8687c8: 279) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868788, args=@0x7f940b8687c8: 279, args=@0x7f940b8687c8: 279) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8687c8: 279, __args=@0x7f940b8687c8: 279) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 626 (Thread 0x7f92c8fb7640 (LWP 2740287) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8deb68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8deb00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8deb00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8deb00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=278) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868760: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868770: 0x7f956f09f800, __args=@0x7f940b868778: 278) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868760: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868778: 278, __args=@0x7f940b868778: 278) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868738, args=@0x7f940b868778: 278, args=@0x7f940b868778: 278) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868778: 278, __args=@0x7f940b868778: 278) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868730) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 625 (Thread 0x7f92c99b8640 (LWP 2740286) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8de868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8de800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8de800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8de800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=277) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868710: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868720: 0x7f956f09f800, __args=@0x7f940b868728: 277) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868710: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868728: 277, __args=@0x7f940b868728: 277) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8686e8, args=@0x7f940b868728: 277, args=@0x7f940b868728: 277) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868728: 277, __args=@0x7f940b868728: 277) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8686e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 624 (Thread 0x7f92ca3b9640 (LWP 2740285) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8de568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8de500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8de500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8de500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=276) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8686c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8686d0: 0x7f956f09f800, __args=@0x7f940b8686d8: 276) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8686c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8686d8: 276, __args=@0x7f940b8686d8: 276) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868698, args=@0x7f940b8686d8: 276, args=@0x7f940b8686d8: 276) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8686d8: 276, __args=@0x7f940b8686d8: 276) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868690) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 623 (Thread 0x7f92cbdba640 (LWP 2740284) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8de268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8de200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8de200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8de200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=275) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868670: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868680: 0x7f956f09f800, __args=@0x7f940b868688: 275) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868670: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868688: 275, __args=@0x7f940b868688: 275) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868648, args=@0x7f940b868688: 275, args=@0x7f940b868688: 275) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868688: 275, __args=@0x7f940b868688: 275) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 622 (Thread 0x7f92cc7bb640 (LWP 2740283) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8ddf68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8ddf00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8ddf00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8ddf00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=274) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868620: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868630: 0x7f956f09f800, __args=@0x7f940b868638: 274) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868620: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868638: 274, __args=@0x7f940b868638: 274) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8685f8, args=@0x7f940b868638: 274, args=@0x7f940b868638: 274) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868638: 274, __args=@0x7f940b868638: 274) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8685f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 621 (Thread 0x7f92cd1bc640 (LWP 2740282) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8ddc68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8ddc00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8ddc00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8ddc00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=273) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8685d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8685e0: 0x7f956f09f800, __args=@0x7f940b8685e8: 273) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8685d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8685e8: 273, __args=@0x7f940b8685e8: 273) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8685a8, args=@0x7f940b8685e8: 273, args=@0x7f940b8685e8: 273) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8685e8: 273, __args=@0x7f940b8685e8: 273) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8685a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 620 (Thread 0x7f92cdbbd640 (LWP 2740281) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dd968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dd900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dd900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dd900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=272) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868580: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868590: 0x7f956f09f800, __args=@0x7f940b868598: 272) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868580: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868598: 272, __args=@0x7f940b868598: 272) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868558, args=@0x7f940b868598: 272, args=@0x7f940b868598: 272) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868598: 272, __args=@0x7f940b868598: 272) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868550) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 619 (Thread 0x7f92ce5be640 (LWP 2740280) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dd668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dd600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dd600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dd600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=271) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868530: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868540: 0x7f956f09f800, __args=@0x7f940b868548: 271) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868530: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868548: 271, __args=@0x7f940b868548: 271) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868508, args=@0x7f940b868548: 271, args=@0x7f940b868548: 271) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868548: 271, __args=@0x7f940b868548: 271) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 618 (Thread 0x7f92cefbf640 (LWP 2740279) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dd368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dd300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dd300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dd300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=270) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8684e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8684f0: 0x7f956f09f800, __args=@0x7f940b8684f8: 270) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8684e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8684f8: 270, __args=@0x7f940b8684f8: 270) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8684b8, args=@0x7f940b8684f8: 270, args=@0x7f940b8684f8: 270) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8684f8: 270, __args=@0x7f940b8684f8: 270) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8684b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 617 (Thread 0x7f92cf9c0640 (LWP 2740278) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dd068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dd000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dd000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dd000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=269) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868490: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8684a0: 0x7f956f09f800, __args=@0x7f940b8684a8: 269) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868490: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8684a8: 269, __args=@0x7f940b8684a8: 269) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868468, args=@0x7f940b8684a8: 269, args=@0x7f940b8684a8: 269) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8684a8: 269, __args=@0x7f940b8684a8: 269) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868460) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 616 (Thread 0x7f92d03c1640 (LWP 2740277) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b857d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b857d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b857d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b857d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=268) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868440: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868450: 0x7f956f09f800, __args=@0x7f940b868458: 268) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868440: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868458: 268, __args=@0x7f940b868458: 268) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868418, args=@0x7f940b868458: 268, args=@0x7f940b868458: 268) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868458: 268, __args=@0x7f940b868458: 268) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868410) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 615 (Thread 0x7f92d0dc2640 (LWP 2740276) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b857a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b857a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b857a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b857a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=267) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8683f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868400: 0x7f956f09f800, __args=@0x7f940b868408: 267) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8683f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868408: 267, __args=@0x7f940b868408: 267) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8683c8, args=@0x7f940b868408: 267, args=@0x7f940b868408: 267) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868408: 267, __args=@0x7f940b868408: 267) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8683c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 614 (Thread 0x7f92d17c3640 (LWP 2740275) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b857768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b857700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b857700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b857700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=266) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8683a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8683b0: 0x7f956f09f800, __args=@0x7f940b8683b8: 266) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8683a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8683b8: 266, __args=@0x7f940b8683b8: 266) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868378, args=@0x7f940b8683b8: 266, args=@0x7f940b8683b8: 266) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8683b8: 266, __args=@0x7f940b8683b8: 266) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868370) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 613 (Thread 0x7f92d21c4640 (LWP 2740274) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b857468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b857400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b857400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b857400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=265) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868350: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868360: 0x7f956f09f800, __args=@0x7f940b868368: 265) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868350: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868368: 265, __args=@0x7f940b868368: 265) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868328, args=@0x7f940b868368: 265, args=@0x7f940b868368: 265) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868368: 265, __args=@0x7f940b868368: 265) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 612 (Thread 0x7f92d2bc5640 (LWP 2740273) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b857168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b857100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b857100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b857100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=264) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868300: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868310: 0x7f956f09f800, __args=@0x7f940b868318: 264) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868300: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868318: 264, __args=@0x7f940b868318: 264) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8682d8, args=@0x7f940b868318: 264, args=@0x7f940b868318: 264) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868318: 264, __args=@0x7f940b868318: 264) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8682d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 611 (Thread 0x7f92d35c6640 (LWP 2740272) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b856e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b856e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b856e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b856e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=263) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8682b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8682c0: 0x7f956f09f800, __args=@0x7f940b8682c8: 263) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8682b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8682c8: 263, __args=@0x7f940b8682c8: 263) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868288, args=@0x7f940b8682c8: 263, args=@0x7f940b8682c8: 263) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8682c8: 263, __args=@0x7f940b8682c8: 263) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 610 (Thread 0x7f92d3fc7640 (LWP 2740271) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b856b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b856b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b856b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b856b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=262) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868260: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868270: 0x7f956f09f800, __args=@0x7f940b868278: 262) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868260: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868278: 262, __args=@0x7f940b868278: 262) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868238, args=@0x7f940b868278: 262, args=@0x7f940b868278: 262) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868278: 262, __args=@0x7f940b868278: 262) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868230) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 609 (Thread 0x7f92d49c8640 (LWP 2740270) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b856868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b856800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b856800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b856800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=261) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868210: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868220: 0x7f956f09f800, __args=@0x7f940b868228: 261) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868210: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868228: 261, __args=@0x7f940b868228: 261) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8681e8, args=@0x7f940b868228: 261, args=@0x7f940b868228: 261) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868228: 261, __args=@0x7f940b868228: 261) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8681e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 608 (Thread 0x7f92d53c9640 (LWP 2740269) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b856568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b856500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b856500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b856500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=260) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8681c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8681d0: 0x7f956f09f800, __args=@0x7f940b8681d8: 260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8681c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8681d8: 260, __args=@0x7f940b8681d8: 260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868198, args=@0x7f940b8681d8: 260, args=@0x7f940b8681d8: 260) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8681d8: 260, __args=@0x7f940b8681d8: 260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868190) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 607 (Thread 0x7f92d5dca640 (LWP 2740268) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b856268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b856200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b856200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b856200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=259) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868170: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868180: 0x7f956f09f800, __args=@0x7f940b868188: 259) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868170: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868188: 259, __args=@0x7f940b868188: 259) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868148, args=@0x7f940b868188: 259, args=@0x7f940b868188: 259) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868188: 259, __args=@0x7f940b868188: 259) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 606 (Thread 0x7f92d71cb640 (LWP 2740267) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b855f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=258) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868120: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868130: 0x7f956f09f800, __args=@0x7f940b868138: 258) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868120: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868138: 258, __args=@0x7f940b868138: 258) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8680f8, args=@0x7f940b868138: 258, args=@0x7f940b868138: 258) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868138: 258, __args=@0x7f940b868138: 258) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8680f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 605 (Thread 0x7f92d7bcc640 (LWP 2740266) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b855c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=257) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8680d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8680e0: 0x7f956f09f800, __args=@0x7f940b8680e8: 257) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8680d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8680e8: 257, __args=@0x7f940b8680e8: 257) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8680a8, args=@0x7f940b8680e8: 257, args=@0x7f940b8680e8: 257) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8680e8: 257, __args=@0x7f940b8680e8: 257) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8680a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 604 (Thread 0x7f92d8fcd640 (LWP 2740265) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b855900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=256) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868080: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868090: 0x7f956f09f800, __args=@0x7f940b868098: 256) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868080: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868098: 256, __args=@0x7f940b868098: 256) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868058, args=@0x7f940b868098: 256, args=@0x7f940b868098: 256) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868098: 256, __args=@0x7f940b868098: 256) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868050) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 603 (Thread 0x7f92d99ce640 (LWP 2740264) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b855600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=255) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868030: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868040: 0x7f956f09f800, __args=@0x7f940b868048: 255) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868030: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868048: 255, __args=@0x7f940b868048: 255) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868008, args=@0x7f940b868048: 255, args=@0x7f940b868048: 255) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868048: 255, __args=@0x7f940b868048: 255) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868000) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 602 (Thread 0x7f92da3cf640 (LWP 2740263) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b855300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=254) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024fe0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024ff0: 0x7f956f09f800, __args=@0x7f940b024ff8: 254) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024fe0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024ff8: 254, __args=@0x7f940b024ff8: 254) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024fb8, args=@0x7f940b024ff8: 254, args=@0x7f940b024ff8: 254) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024ff8: 254, __args=@0x7f940b024ff8: 254) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024fb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 601 (Thread 0x7f92dadd0640 (LWP 2740262) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b855000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=253) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024f90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024fa0: 0x7f956f09f800, __args=@0x7f940b024fa8: 253) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024f90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024fa8: 253, __args=@0x7f940b024fa8: 253) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024f68, args=@0x7f940b024fa8: 253, args=@0x7f940b024fa8: 253) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024fa8: 253, __args=@0x7f940b024fa8: 253) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024f60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 600 (Thread 0x7f92db7d1640 (LWP 2740261) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d4d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d4d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d4d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d4d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=252) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024f40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024f50: 0x7f956f09f800, __args=@0x7f940b024f58: 252) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024f40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024f58: 252, __args=@0x7f940b024f58: 252) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024f18, args=@0x7f940b024f58: 252, args=@0x7f940b024f58: 252) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024f58: 252, __args=@0x7f940b024f58: 252) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024f10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 599 (Thread 0x7f92dc1d2640 (LWP 2740260) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d4a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d4a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d4a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d4a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=251) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ef0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024f00: 0x7f956f09f800, __args=@0x7f940b024f08: 251) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ef0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024f08: 251, __args=@0x7f940b024f08: 251) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024ec8, args=@0x7f940b024f08: 251, args=@0x7f940b024f08: 251) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024f08: 251, __args=@0x7f940b024f08: 251) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024ec0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 598 (Thread 0x7f92dcbd3640 (LWP 2740259) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d4768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d4700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d4700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d4700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=250) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ea0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024eb0: 0x7f956f09f800, __args=@0x7f940b024eb8: 250) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ea0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024eb8: 250, __args=@0x7f940b024eb8: 250) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024e78, args=@0x7f940b024eb8: 250, args=@0x7f940b024eb8: 250) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024eb8: 250, __args=@0x7f940b024eb8: 250) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024e70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 597 (Thread 0x7f92dd5d4640 (LWP 2740258) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d4468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d4400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d4400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d4400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=249) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024e50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024e60: 0x7f956f09f800, __args=@0x7f940b024e68: 249) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024e50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024e68: 249, __args=@0x7f940b024e68: 249) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024e28, args=@0x7f940b024e68: 249, args=@0x7f940b024e68: 249) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024e68: 249, __args=@0x7f940b024e68: 249) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024e20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 596 (Thread 0x7f92ddfd5640 (LWP 2740257) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d4168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d4100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d4100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d4100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=248) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024e00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024e10: 0x7f956f09f800, __args=@0x7f940b024e18: 248) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024e00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024e18: 248, __args=@0x7f940b024e18: 248) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024dd8, args=@0x7f940b024e18: 248, args=@0x7f940b024e18: 248) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024e18: 248, __args=@0x7f940b024e18: 248) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024dd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 595 (Thread 0x7f92de9d6640 (LWP 2740256) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d3e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d3e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d3e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d3e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=247) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024db0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024dc0: 0x7f956f09f800, __args=@0x7f940b024dc8: 247) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024db0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024dc8: 247, __args=@0x7f940b024dc8: 247) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024d88, args=@0x7f940b024dc8: 247, args=@0x7f940b024dc8: 247) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024dc8: 247, __args=@0x7f940b024dc8: 247) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024d80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 594 (Thread 0x7f92df3d7640 (LWP 2740255) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d3b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d3b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d3b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d3b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=246) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024d60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024d70: 0x7f956f09f800, __args=@0x7f940b024d78: 246) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024d60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024d78: 246, __args=@0x7f940b024d78: 246) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024d38, args=@0x7f940b024d78: 246, args=@0x7f940b024d78: 246) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024d78: 246, __args=@0x7f940b024d78: 246) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024d30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 593 (Thread 0x7f92dfdd8640 (LWP 2740254) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d3868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d3800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d3800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d3800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=245) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024d10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024d20: 0x7f956f09f800, __args=@0x7f940b024d28: 245) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024d10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024d28: 245, __args=@0x7f940b024d28: 245) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024ce8, args=@0x7f940b024d28: 245, args=@0x7f940b024d28: 245) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024d28: 245, __args=@0x7f940b024d28: 245) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024ce0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 592 (Thread 0x7f92e07d9640 (LWP 2740253) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d3568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d3500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d3500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d3500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=244) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024cc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024cd0: 0x7f956f09f800, __args=@0x7f940b024cd8: 244) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024cc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024cd8: 244, __args=@0x7f940b024cd8: 244) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024c98, args=@0x7f940b024cd8: 244, args=@0x7f940b024cd8: 244) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024cd8: 244, __args=@0x7f940b024cd8: 244) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024c90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 591 (Thread 0x7f92e11da640 (LWP 2740252) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d3268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d3200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d3200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d3200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=243) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024c70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024c80: 0x7f956f09f800, __args=@0x7f940b024c88: 243) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024c70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024c88: 243, __args=@0x7f940b024c88: 243) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024c48, args=@0x7f940b024c88: 243, args=@0x7f940b024c88: 243) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024c88: 243, __args=@0x7f940b024c88: 243) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 590 (Thread 0x7f92e1bdb640 (LWP 2740251) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=242) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024c20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024c30: 0x7f956f09f800, __args=@0x7f940b024c38: 242) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024c20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024c38: 242, __args=@0x7f940b024c38: 242) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024bf8, args=@0x7f940b024c38: 242, args=@0x7f940b024c38: 242) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024c38: 242, __args=@0x7f940b024c38: 242) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024bf0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 589 (Thread 0x7f92e25dc640 (LWP 2740250) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=241) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024bd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024be0: 0x7f956f09f800, __args=@0x7f940b024be8: 241) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024bd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024be8: 241, __args=@0x7f940b024be8: 241) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024ba8, args=@0x7f940b024be8: 241, args=@0x7f940b024be8: 241) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024be8: 241, __args=@0x7f940b024be8: 241) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 588 (Thread 0x7f92e2fdd640 (LWP 2740249) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=240) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024b80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024b90: 0x7f956f09f800, __args=@0x7f940b024b98: 240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024b80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024b98: 240, __args=@0x7f940b024b98: 240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024b58, args=@0x7f940b024b98: 240, args=@0x7f940b024b98: 240) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024b98: 240, __args=@0x7f940b024b98: 240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024b50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 587 (Thread 0x7f92e39de640 (LWP 2740248) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=239) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024b30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024b40: 0x7f956f09f800, __args=@0x7f940b024b48: 239) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024b30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024b48: 239, __args=@0x7f940b024b48: 239) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024b08, args=@0x7f940b024b48: 239, args=@0x7f940b024b48: 239) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024b48: 239, __args=@0x7f940b024b48: 239) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024b00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 586 (Thread 0x7f92e4ddf640 (LWP 2740247) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=238) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ae0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024af0: 0x7f956f09f800, __args=@0x7f940b024af8: 238) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ae0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024af8: 238, __args=@0x7f940b024af8: 238) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024ab8, args=@0x7f940b024af8: 238, args=@0x7f940b024af8: 238) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024af8: 238, __args=@0x7f940b024af8: 238) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024ab0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 585 (Thread 0x7f92e57e0640 (LWP 2740246) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=237) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024a90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024aa0: 0x7f956f09f800, __args=@0x7f940b024aa8: 237) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024a90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024aa8: 237, __args=@0x7f940b024aa8: 237) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024a68, args=@0x7f940b024aa8: 237, args=@0x7f940b024aa8: 237) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024aa8: 237, __args=@0x7f940b024aa8: 237) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024a60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 584 (Thread 0x7f92e61e1640 (LWP 2740245) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b751d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b751d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b751d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b751d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=236) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024a40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024a50: 0x7f956f09f800, __args=@0x7f940b024a58: 236) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024a40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024a58: 236, __args=@0x7f940b024a58: 236) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024a18, args=@0x7f940b024a58: 236, args=@0x7f940b024a58: 236) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024a58: 236, __args=@0x7f940b024a58: 236) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024a10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 583 (Thread 0x7f92e6be2640 (LWP 2740244) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b751a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b751a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b751a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b751a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=235) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0249f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024a00: 0x7f956f09f800, __args=@0x7f940b024a08: 235) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0249f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024a08: 235, __args=@0x7f940b024a08: 235) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0249c8, args=@0x7f940b024a08: 235, args=@0x7f940b024a08: 235) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024a08: 235, __args=@0x7f940b024a08: 235) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0249c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 582 (Thread 0x7f92e75e3640 (LWP 2740243) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b751768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b751700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b751700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b751700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=234) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0249a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0249b0: 0x7f956f09f800, __args=@0x7f940b0249b8: 234) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0249a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0249b8: 234, __args=@0x7f940b0249b8: 234) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024978, args=@0x7f940b0249b8: 234, args=@0x7f940b0249b8: 234) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0249b8: 234, __args=@0x7f940b0249b8: 234) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024970) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 581 (Thread 0x7f92e7fe4640 (LWP 2740242) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b751468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b751400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b751400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b751400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=233) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024950: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024960: 0x7f956f09f800, __args=@0x7f940b024968: 233) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024950: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024968: 233, __args=@0x7f940b024968: 233) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024928, args=@0x7f940b024968: 233, args=@0x7f940b024968: 233) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024968: 233, __args=@0x7f940b024968: 233) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024920) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 580 (Thread 0x7f92e89e5640 (LWP 2740241) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b751168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b751100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b751100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b751100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=232) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024900: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024910: 0x7f956f09f800, __args=@0x7f940b024918: 232) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024900: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024918: 232, __args=@0x7f940b024918: 232) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0248d8, args=@0x7f940b024918: 232, args=@0x7f940b024918: 232) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024918: 232, __args=@0x7f940b024918: 232) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0248d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 579 (Thread 0x7f92e93e6640 (LWP 2740240) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b750e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b750e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b750e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b750e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=231) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0248b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0248c0: 0x7f956f09f800, __args=@0x7f940b0248c8: 231) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0248b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0248c8: 231, __args=@0x7f940b0248c8: 231) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024888, args=@0x7f940b0248c8: 231, args=@0x7f940b0248c8: 231) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0248c8: 231, __args=@0x7f940b0248c8: 231) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024880) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 578 (Thread 0x7f92e9de7640 (LWP 2740239) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b750b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b750b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b750b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b750b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=230) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024860: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024870: 0x7f956f09f800, __args=@0x7f940b024878: 230) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024860: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024878: 230, __args=@0x7f940b024878: 230) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024838, args=@0x7f940b024878: 230, args=@0x7f940b024878: 230) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024878: 230, __args=@0x7f940b024878: 230) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024830) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 577 (Thread 0x7f92ea7e8640 (LWP 2740238) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b750868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b750800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b750800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b750800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=229) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024810: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024820: 0x7f956f09f800, __args=@0x7f940b024828: 229) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024810: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024828: 229, __args=@0x7f940b024828: 229) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0247e8, args=@0x7f940b024828: 229, args=@0x7f940b024828: 229) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024828: 229, __args=@0x7f940b024828: 229) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0247e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 576 (Thread 0x7f92ebbe9640 (LWP 2740237) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b750568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b750500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b750500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b750500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=228) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0247c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0247d0: 0x7f956f09f800, __args=@0x7f940b0247d8: 228) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0247c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0247d8: 228, __args=@0x7f940b0247d8: 228) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024798, args=@0x7f940b0247d8: 228, args=@0x7f940b0247d8: 228) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0247d8: 228, __args=@0x7f940b0247d8: 228) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024790) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 575 (Thread 0x7f92ec5ea640 (LWP 2740236) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b750268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b750200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b750200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b750200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=227) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024770: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024780: 0x7f956f09f800, __args=@0x7f940b024788: 227) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024770: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024788: 227, __args=@0x7f940b024788: 227) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024748, args=@0x7f940b024788: 227, args=@0x7f940b024788: 227) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024788: 227, __args=@0x7f940b024788: 227) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 574 (Thread 0x7f92ecfeb640 (LWP 2740235) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74ff68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74ff00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b74ff00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74ff00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=226) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024720: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024730: 0x7f956f09f800, __args=@0x7f940b024738: 226) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024720: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024738: 226, __args=@0x7f940b024738: 226) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0246f8, args=@0x7f940b024738: 226, args=@0x7f940b024738: 226) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024738: 226, __args=@0x7f940b024738: 226) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0246f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 573 (Thread 0x7f92ed9ec640 (LWP 2740234) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74fc68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74fc00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b74fc00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74fc00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=225) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0246d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0246e0: 0x7f956f09f800, __args=@0x7f940b0246e8: 225) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0246d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0246e8: 225, __args=@0x7f940b0246e8: 225) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0246a8, args=@0x7f940b0246e8: 225, args=@0x7f940b0246e8: 225) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0246e8: 225, __args=@0x7f940b0246e8: 225) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0246a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 572 (Thread 0x7f92ee3ed640 (LWP 2740233) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74f968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74f900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b74f900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74f900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=224) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024680: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024690: 0x7f956f09f800, __args=@0x7f940b024698: 224) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024680: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024698: 224, __args=@0x7f940b024698: 224) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024658, args=@0x7f940b024698: 224, args=@0x7f940b024698: 224) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024698: 224, __args=@0x7f940b024698: 224) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024650) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 571 (Thread 0x7f92eedee640 (LWP 2740232) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74f668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74f600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b74f600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74f600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=223) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024630: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024640: 0x7f956f09f800, __args=@0x7f940b024648: 223) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024630: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024648: 223, __args=@0x7f940b024648: 223) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024608, args=@0x7f940b024648: 223, args=@0x7f940b024648: 223) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024648: 223, __args=@0x7f940b024648: 223) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 570 (Thread 0x7f92ef7ef640 (LWP 2740231) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74f368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74f300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b74f300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74f300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=222) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0245e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0245f0: 0x7f956f09f800, __args=@0x7f940b0245f8: 222) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0245e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0245f8: 222, __args=@0x7f940b0245f8: 222) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0245b8, args=@0x7f940b0245f8: 222, args=@0x7f940b0245f8: 222) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0245f8: 222, __args=@0x7f940b0245f8: 222) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0245b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 569 (Thread 0x7f92f0bf0640 (LWP 2740230) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74f068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74f000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b74f000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74f000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=221) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024590: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0245a0: 0x7f956f09f800, __args=@0x7f940b0245a8: 221) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024590: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0245a8: 221, __args=@0x7f940b0245a8: 221) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024568, args=@0x7f940b0245a8: 221, args=@0x7f940b0245a8: 221) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0245a8: 221, __args=@0x7f940b0245a8: 221) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024560) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 568 (Thread 0x7f92f1ff1640 (LWP 2740229) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ced68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ced00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ced00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ced00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=220) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024540: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024550: 0x7f956f09f800, __args=@0x7f940b024558: 220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024540: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024558: 220, __args=@0x7f940b024558: 220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024518, args=@0x7f940b024558: 220, args=@0x7f940b024558: 220) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024558: 220, __args=@0x7f940b024558: 220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024510) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 567 (Thread 0x7f92f29f2640 (LWP 2740228) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cea68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cea00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cea00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cea00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=219) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0244f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024500: 0x7f956f09f800, __args=@0x7f940b024508: 219) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0244f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024508: 219, __args=@0x7f940b024508: 219) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0244c8, args=@0x7f940b024508: 219, args=@0x7f940b024508: 219) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024508: 219, __args=@0x7f940b024508: 219) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0244c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 566 (Thread 0x7f92f33f3640 (LWP 2740227) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ce768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ce700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ce700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ce700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=218) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0244a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0244b0: 0x7f956f09f800, __args=@0x7f940b0244b8: 218) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0244a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0244b8: 218, __args=@0x7f940b0244b8: 218) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024478, args=@0x7f940b0244b8: 218, args=@0x7f940b0244b8: 218) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0244b8: 218, __args=@0x7f940b0244b8: 218) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024470) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 565 (Thread 0x7f92f3df4640 (LWP 2740226) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ce468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ce400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ce400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ce400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=217) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024450: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024460: 0x7f956f09f800, __args=@0x7f940b024468: 217) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024450: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024468: 217, __args=@0x7f940b024468: 217) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024428, args=@0x7f940b024468: 217, args=@0x7f940b024468: 217) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024468: 217, __args=@0x7f940b024468: 217) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024420) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 564 (Thread 0x7f92f47f5640 (LWP 2740225) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ce168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ce100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ce100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ce100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=216) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024400: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024410: 0x7f956f09f800, __args=@0x7f940b024418: 216) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024400: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024418: 216, __args=@0x7f940b024418: 216) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0243d8, args=@0x7f940b024418: 216, args=@0x7f940b024418: 216) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024418: 216, __args=@0x7f940b024418: 216) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0243d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 563 (Thread 0x7f92f51f6640 (LWP 2740224) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cde68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cde00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cde00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cde00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=215) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0243b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0243c0: 0x7f956f09f800, __args=@0x7f940b0243c8: 215) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0243b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0243c8: 215, __args=@0x7f940b0243c8: 215) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024388, args=@0x7f940b0243c8: 215, args=@0x7f940b0243c8: 215) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0243c8: 215, __args=@0x7f940b0243c8: 215) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 562 (Thread 0x7f92f5bf7640 (LWP 2740223) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cdb68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cdb00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cdb00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cdb00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=214) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024360: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024370: 0x7f956f09f800, __args=@0x7f940b024378: 214) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024360: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024378: 214, __args=@0x7f940b024378: 214) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024338, args=@0x7f940b024378: 214, args=@0x7f940b024378: 214) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024378: 214, __args=@0x7f940b024378: 214) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024330) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 561 (Thread 0x7f92f65f8640 (LWP 2740222) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cd868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cd800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cd800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cd800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=213) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024310: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024320: 0x7f956f09f800, __args=@0x7f940b024328: 213) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024310: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024328: 213, __args=@0x7f940b024328: 213) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0242e8, args=@0x7f940b024328: 213, args=@0x7f940b024328: 213) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024328: 213, __args=@0x7f940b024328: 213) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0242e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 560 (Thread 0x7f92f6ff9640 (LWP 2740221) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cd568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cd500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cd500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cd500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=212) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0242c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0242d0: 0x7f956f09f800, __args=@0x7f940b0242d8: 212) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0242c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0242d8: 212, __args=@0x7f940b0242d8: 212) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024298, args=@0x7f940b0242d8: 212, args=@0x7f940b0242d8: 212) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0242d8: 212, __args=@0x7f940b0242d8: 212) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024290) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 559 (Thread 0x7f92f79fa640 (LWP 2740220) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cd268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cd200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cd200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cd200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=211) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024270: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024280: 0x7f956f09f800, __args=@0x7f940b024288: 211) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024270: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024288: 211, __args=@0x7f940b024288: 211) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024248, args=@0x7f940b024288: 211, args=@0x7f940b024288: 211) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024288: 211, __args=@0x7f940b024288: 211) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 558 (Thread 0x7f92f83fb640 (LWP 2740219) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ccf68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ccf00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ccf00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ccf00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=210) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024220: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024230: 0x7f956f09f800, __args=@0x7f940b024238: 210) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024220: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024238: 210, __args=@0x7f940b024238: 210) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0241f8, args=@0x7f940b024238: 210, args=@0x7f940b024238: 210) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024238: 210, __args=@0x7f940b024238: 210) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0241f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 557 (Thread 0x7f92f8dfc640 (LWP 2740218) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ccc68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ccc00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ccc00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ccc00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=209) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0241d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0241e0: 0x7f956f09f800, __args=@0x7f940b0241e8: 209) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0241d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0241e8: 209, __args=@0x7f940b0241e8: 209) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0241a8, args=@0x7f940b0241e8: 209, args=@0x7f940b0241e8: 209) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0241e8: 209, __args=@0x7f940b0241e8: 209) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0241a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 556 (Thread 0x7f92f97fd640 (LWP 2740217) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cc968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cc900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cc900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cc900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=208) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024180: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024190: 0x7f956f09f800, __args=@0x7f940b024198: 208) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024180: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024198: 208, __args=@0x7f940b024198: 208) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024158, args=@0x7f940b024198: 208, args=@0x7f940b024198: 208) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024198: 208, __args=@0x7f940b024198: 208) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024150) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 555 (Thread 0x7f92fa1fe640 (LWP 2740216) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cc668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cc600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cc600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cc600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=207) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024130: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024140: 0x7f956f09f800, __args=@0x7f940b024148: 207) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024130: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024148: 207, __args=@0x7f940b024148: 207) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024108, args=@0x7f940b024148: 207, args=@0x7f940b024148: 207) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024148: 207, __args=@0x7f940b024148: 207) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 554 (Thread 0x7f92fabff640 (LWP 2740215) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cc368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cc300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cc300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cc300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=206) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0240e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0240f0: 0x7f956f09f800, __args=@0x7f940b0240f8: 206) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0240e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0240f8: 206, __args=@0x7f940b0240f8: 206) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0240b8, args=@0x7f940b0240f8: 206, args=@0x7f940b0240f8: 206) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0240f8: 206, __args=@0x7f940b0240f8: 206) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0240b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 553 (Thread 0x7f92fcca0640 (LWP 2740214) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cc068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cc000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cc000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cc000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=205) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024090: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0240a0: 0x7f956f09f800, __args=@0x7f940b0240a8: 205) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024090: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0240a8: 205, __args=@0x7f940b0240a8: 205) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024068, args=@0x7f940b0240a8: 205, args=@0x7f940b0240a8: 205) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0240a8: 205, __args=@0x7f940b0240a8: 205) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024060) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 552 (Thread 0x7f92fe0a1640 (LWP 2740213) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64bd68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64bd00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b64bd00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64bd00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=204) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024040: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024050: 0x7f956f09f800, __args=@0x7f940b024058: 204) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024040: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024058: 204, __args=@0x7f940b024058: 204) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024018, args=@0x7f940b024058: 204, args=@0x7f940b024058: 204) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024058: 204, __args=@0x7f940b024058: 204) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024010) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 551 (Thread 0x7f92feaa2640 (LWP 2740212) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64ba68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64ba00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b64ba00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64ba00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=203) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023ff0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024000: 0x7f956f09f800, __args=@0x7f940b024008: 203) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023ff0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024008: 203, __args=@0x7f940b024008: 203) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023fc8, args=@0x7f940b024008: 203, args=@0x7f940b024008: 203) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024008: 203, __args=@0x7f940b024008: 203) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023fc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 550 (Thread 0x7f92ff4a3640 (LWP 2740211) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64b768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64b700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b64b700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64b700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=202) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023fa0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023fb0: 0x7f956f09f800, __args=@0x7f940b023fb8: 202) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023fa0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023fb8: 202, __args=@0x7f940b023fb8: 202) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023f78, args=@0x7f940b023fb8: 202, args=@0x7f940b023fb8: 202) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023fb8: 202, __args=@0x7f940b023fb8: 202) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023f70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 549 (Thread 0x7f92ffea4640 (LWP 2740210) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64b468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64b400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b64b400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64b400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=201) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023f50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023f60: 0x7f956f09f800, __args=@0x7f940b023f68: 201) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023f50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023f68: 201, __args=@0x7f940b023f68: 201) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023f28, args=@0x7f940b023f68: 201, args=@0x7f940b023f68: 201) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023f68: 201, __args=@0x7f940b023f68: 201) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023f20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 548 (Thread 0x7f93008a5640 (LWP 2740209) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64b168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64b100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b64b100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64b100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=200) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023f00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023f10: 0x7f956f09f800, __args=@0x7f940b023f18: 200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023f00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023f18: 200, __args=@0x7f940b023f18: 200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023ed8, args=@0x7f940b023f18: 200, args=@0x7f940b023f18: 200) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023f18: 200, __args=@0x7f940b023f18: 200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023ed0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 547 (Thread 0x7f93012a6640 (LWP 2740208) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64ae68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64ae00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b64ae00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64ae00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=199) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023eb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023ec0: 0x7f956f09f800, __args=@0x7f940b023ec8: 199) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023eb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023ec8: 199, __args=@0x7f940b023ec8: 199) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023e88, args=@0x7f940b023ec8: 199, args=@0x7f940b023ec8: 199) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023ec8: 199, __args=@0x7f940b023ec8: 199) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023e80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 546 (Thread 0x7f9301ca7640 (LWP 2740207) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64ab68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64ab00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b64ab00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64ab00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=198) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023e60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023e70: 0x7f956f09f800, __args=@0x7f940b023e78: 198) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023e60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023e78: 198, __args=@0x7f940b023e78: 198) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023e38, args=@0x7f940b023e78: 198, args=@0x7f940b023e78: 198) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023e78: 198, __args=@0x7f940b023e78: 198) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023e30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 545 (Thread 0x7f93026a8640 (LWP 2740206) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64a868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64a800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b64a800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64a800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=197) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023e10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023e20: 0x7f956f09f800, __args=@0x7f940b023e28: 197) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023e10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023e28: 197, __args=@0x7f940b023e28: 197) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023de8, args=@0x7f940b023e28: 197, args=@0x7f940b023e28: 197) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023e28: 197, __args=@0x7f940b023e28: 197) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023de0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 544 (Thread 0x7f93030a9640 (LWP 2740205) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64a568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64a500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b64a500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64a500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=196) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023dc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023dd0: 0x7f956f09f800, __args=@0x7f940b023dd8: 196) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023dc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023dd8: 196, __args=@0x7f940b023dd8: 196) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023d98, args=@0x7f940b023dd8: 196, args=@0x7f940b023dd8: 196) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023dd8: 196, __args=@0x7f940b023dd8: 196) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023d90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 543 (Thread 0x7f9303aaa640 (LWP 2740204) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64a268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64a200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b64a200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64a200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=195) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023d70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023d80: 0x7f956f09f800, __args=@0x7f940b023d88: 195) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023d70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023d88: 195, __args=@0x7f940b023d88: 195) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023d48, args=@0x7f940b023d88: 195, args=@0x7f940b023d88: 195) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023d88: 195, __args=@0x7f940b023d88: 195) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023d40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 542 (Thread 0x7f93044ab640 (LWP 2740203) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b649f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=194) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023d20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023d30: 0x7f956f09f800, __args=@0x7f940b023d38: 194) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023d20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023d38: 194, __args=@0x7f940b023d38: 194) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023cf8, args=@0x7f940b023d38: 194, args=@0x7f940b023d38: 194) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023d38: 194, __args=@0x7f940b023d38: 194) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023cf0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 541 (Thread 0x7f9304eac640 (LWP 2740202) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b649c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=193) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023cd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023ce0: 0x7f956f09f800, __args=@0x7f940b023ce8: 193) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023cd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023ce8: 193, __args=@0x7f940b023ce8: 193) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023ca8, args=@0x7f940b023ce8: 193, args=@0x7f940b023ce8: 193) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023ce8: 193, __args=@0x7f940b023ce8: 193) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023ca0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 540 (Thread 0x7f93058ad640 (LWP 2740201) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b649900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=192) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023c80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023c90: 0x7f956f09f800, __args=@0x7f940b023c98: 192) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023c80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023c98: 192, __args=@0x7f940b023c98: 192) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023c58, args=@0x7f940b023c98: 192, args=@0x7f940b023c98: 192) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023c98: 192, __args=@0x7f940b023c98: 192) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023c50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 539 (Thread 0x7f93062ae640 (LWP 2740200) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b649600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=191) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023c30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023c40: 0x7f956f09f800, __args=@0x7f940b023c48: 191) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023c30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023c48: 191, __args=@0x7f940b023c48: 191) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023c08, args=@0x7f940b023c48: 191, args=@0x7f940b023c48: 191) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023c48: 191, __args=@0x7f940b023c48: 191) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023c00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 538 (Thread 0x7f9306caf640 (LWP 2740199) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b649300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=190) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023be0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023bf0: 0x7f956f09f800, __args=@0x7f940b023bf8: 190) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023be0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023bf8: 190, __args=@0x7f940b023bf8: 190) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023bb8, args=@0x7f940b023bf8: 190, args=@0x7f940b023bf8: 190) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023bf8: 190, __args=@0x7f940b023bf8: 190) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023bb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 537 (Thread 0x7f93076b0640 (LWP 2740198) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b649000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=189) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023b90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023ba0: 0x7f956f09f800, __args=@0x7f940b023ba8: 189) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023b90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023ba8: 189, __args=@0x7f940b023ba8: 189) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023b68, args=@0x7f940b023ba8: 189, args=@0x7f940b023ba8: 189) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023ba8: 189, __args=@0x7f940b023ba8: 189) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023b60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 536 (Thread 0x7f93080b1640 (LWP 2740197) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c8d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c8d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c8d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c8d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=188) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023b40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023b50: 0x7f956f09f800, __args=@0x7f940b023b58: 188) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023b40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023b58: 188, __args=@0x7f940b023b58: 188) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023b18, args=@0x7f940b023b58: 188, args=@0x7f940b023b58: 188) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023b58: 188, __args=@0x7f940b023b58: 188) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023b10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 535 (Thread 0x7f9308ab2640 (LWP 2740196) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c8a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c8a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c8a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c8a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=187) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023af0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023b00: 0x7f956f09f800, __args=@0x7f940b023b08: 187) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023af0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023b08: 187, __args=@0x7f940b023b08: 187) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023ac8, args=@0x7f940b023b08: 187, args=@0x7f940b023b08: 187) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023b08: 187, __args=@0x7f940b023b08: 187) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023ac0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 534 (Thread 0x7f93094b3640 (LWP 2740195) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c8768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c8700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c8700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c8700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=186) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023aa0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023ab0: 0x7f956f09f800, __args=@0x7f940b023ab8: 186) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023aa0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023ab8: 186, __args=@0x7f940b023ab8: 186) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023a78, args=@0x7f940b023ab8: 186, args=@0x7f940b023ab8: 186) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023ab8: 186, __args=@0x7f940b023ab8: 186) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023a70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 533 (Thread 0x7f9309eb4640 (LWP 2740194) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c8468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c8400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c8400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c8400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=185) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023a50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023a60: 0x7f956f09f800, __args=@0x7f940b023a68: 185) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023a50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023a68: 185, __args=@0x7f940b023a68: 185) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023a28, args=@0x7f940b023a68: 185, args=@0x7f940b023a68: 185) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023a68: 185, __args=@0x7f940b023a68: 185) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023a20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 532 (Thread 0x7f930a8b5640 (LWP 2740193) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c8168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c8100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c8100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c8100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=184) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023a00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023a10: 0x7f956f09f800, __args=@0x7f940b023a18: 184) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023a00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023a18: 184, __args=@0x7f940b023a18: 184) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0239d8, args=@0x7f940b023a18: 184, args=@0x7f940b023a18: 184) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023a18: 184, __args=@0x7f940b023a18: 184) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0239d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 531 (Thread 0x7f930b2b6640 (LWP 2740192) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c7e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c7e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c7e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c7e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=183) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0239b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0239c0: 0x7f956f09f800, __args=@0x7f940b0239c8: 183) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0239b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0239c8: 183, __args=@0x7f940b0239c8: 183) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023988, args=@0x7f940b0239c8: 183, args=@0x7f940b0239c8: 183) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0239c8: 183, __args=@0x7f940b0239c8: 183) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023980) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 530 (Thread 0x7f930c6b7640 (LWP 2740191) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c7b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c7b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c7b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c7b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=182) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023960: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023970: 0x7f956f09f800, __args=@0x7f940b023978: 182) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023960: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023978: 182, __args=@0x7f940b023978: 182) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023938, args=@0x7f940b023978: 182, args=@0x7f940b023978: 182) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023978: 182, __args=@0x7f940b023978: 182) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023930) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 529 (Thread 0x7f930d0b8640 (LWP 2740190) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c7868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c7800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c7800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c7800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=181) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023910: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023920: 0x7f956f09f800, __args=@0x7f940b023928: 181) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023910: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023928: 181, __args=@0x7f940b023928: 181) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0238e8, args=@0x7f940b023928: 181, args=@0x7f940b023928: 181) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023928: 181, __args=@0x7f940b023928: 181) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0238e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 528 (Thread 0x7f930dab9640 (LWP 2740189) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c7568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c7500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c7500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c7500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=180) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0238c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0238d0: 0x7f956f09f800, __args=@0x7f940b0238d8: 180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0238c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0238d8: 180, __args=@0x7f940b0238d8: 180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023898, args=@0x7f940b0238d8: 180, args=@0x7f940b0238d8: 180) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0238d8: 180, __args=@0x7f940b0238d8: 180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023890) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 527 (Thread 0x7f930e4ba640 (LWP 2740188) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c7268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c7200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c7200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c7200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=179) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023870: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023880: 0x7f956f09f800, __args=@0x7f940b023888: 179) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023870: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023888: 179, __args=@0x7f940b023888: 179) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023848, args=@0x7f940b023888: 179, args=@0x7f940b023888: 179) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023888: 179, __args=@0x7f940b023888: 179) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 526 (Thread 0x7f930eebb640 (LWP 2740187) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=178) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023820: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023830: 0x7f956f09f800, __args=@0x7f940b023838: 178) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023820: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023838: 178, __args=@0x7f940b023838: 178) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0237f8, args=@0x7f940b023838: 178, args=@0x7f940b023838: 178) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023838: 178, __args=@0x7f940b023838: 178) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0237f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 525 (Thread 0x7f930fdbc640 (LWP 2740186) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=177) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0237d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0237e0: 0x7f956f09f800, __args=@0x7f940b0237e8: 177) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0237d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0237e8: 177, __args=@0x7f940b0237e8: 177) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0237a8, args=@0x7f940b0237e8: 177, args=@0x7f940b0237e8: 177) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0237e8: 177, __args=@0x7f940b0237e8: 177) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0237a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 524 (Thread 0x7f93107bd640 (LWP 2740185) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=176) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023780: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023790: 0x7f956f09f800, __args=@0x7f940b023798: 176) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023780: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023798: 176, __args=@0x7f940b023798: 176) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023758, args=@0x7f940b023798: 176, args=@0x7f940b023798: 176) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023798: 176, __args=@0x7f940b023798: 176) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023750) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 523 (Thread 0x7f93111be640 (LWP 2740184) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=175) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023730: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023740: 0x7f956f09f800, __args=@0x7f940b023748: 175) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023730: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023748: 175, __args=@0x7f940b023748: 175) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023708, args=@0x7f940b023748: 175, args=@0x7f940b023748: 175) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023748: 175, __args=@0x7f940b023748: 175) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023700) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 522 (Thread 0x7f9311bbf640 (LWP 2740183) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=174) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0236e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0236f0: 0x7f956f09f800, __args=@0x7f940b0236f8: 174) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0236e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0236f8: 174, __args=@0x7f940b0236f8: 174) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0236b8, args=@0x7f940b0236f8: 174, args=@0x7f940b0236f8: 174) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0236f8: 174, __args=@0x7f940b0236f8: 174) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0236b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 521 (Thread 0x7f93125c0640 (LWP 2740182) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=173) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023690: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0236a0: 0x7f956f09f800, __args=@0x7f940b0236a8: 173) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023690: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0236a8: 173, __args=@0x7f940b0236a8: 173) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023668, args=@0x7f940b0236a8: 173, args=@0x7f940b0236a8: 173) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0236a8: 173, __args=@0x7f940b0236a8: 173) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023660) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 520 (Thread 0x7f9312fc1640 (LWP 2740181) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b545d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b545d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b545d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b545d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=172) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023640: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023650: 0x7f956f09f800, __args=@0x7f940b023658: 172) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023640: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023658: 172, __args=@0x7f940b023658: 172) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023618, args=@0x7f940b023658: 172, args=@0x7f940b023658: 172) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023658: 172, __args=@0x7f940b023658: 172) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023610) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 519 (Thread 0x7f93139c2640 (LWP 2740180) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b545a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b545a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b545a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b545a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=171) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0235f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023600: 0x7f956f09f800, __args=@0x7f940b023608: 171) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0235f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023608: 171, __args=@0x7f940b023608: 171) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0235c8, args=@0x7f940b023608: 171, args=@0x7f940b023608: 171) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023608: 171, __args=@0x7f940b023608: 171) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0235c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 518 (Thread 0x7f93143c3640 (LWP 2740179) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b545768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b545700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b545700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b545700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=170) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0235a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0235b0: 0x7f956f09f800, __args=@0x7f940b0235b8: 170) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0235a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0235b8: 170, __args=@0x7f940b0235b8: 170) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023578, args=@0x7f940b0235b8: 170, args=@0x7f940b0235b8: 170) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0235b8: 170, __args=@0x7f940b0235b8: 170) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023570) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 517 (Thread 0x7f9314dc4640 (LWP 2740177) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b545468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b545400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b545400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b545400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=169) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023550: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023560: 0x7f956f09f800, __args=@0x7f940b023568: 169) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023550: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023568: 169, __args=@0x7f940b023568: 169) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023528, args=@0x7f940b023568: 169, args=@0x7f940b023568: 169) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023568: 169, __args=@0x7f940b023568: 169) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023520) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 516 (Thread 0x7f93157c5640 (LWP 2740176) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b545168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b545100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b545100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b545100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023500: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023510: 0x7f956f09f800, __args=@0x7f940b023518: 168) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023500: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023518: 168, __args=@0x7f940b023518: 168) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0234d8, args=@0x7f940b023518: 168, args=@0x7f940b023518: 168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023518: 168, __args=@0x7f940b023518: 168) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0234d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 515 (Thread 0x7f93161c6640 (LWP 2740174) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b544e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b544e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b544e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b544e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=167) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0234b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0234c0: 0x7f956f09f800, __args=@0x7f940b0234c8: 167) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0234b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0234c8: 167, __args=@0x7f940b0234c8: 167) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023488, args=@0x7f940b0234c8: 167, args=@0x7f940b0234c8: 167) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0234c8: 167, __args=@0x7f940b0234c8: 167) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 514 (Thread 0x7f93171c7640 (LWP 2740172) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b544b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b544b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b544b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b544b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=166) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023460: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023470: 0x7f956f09f800, __args=@0x7f940b023478: 166) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023460: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023478: 166, __args=@0x7f940b023478: 166) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023438, args=@0x7f940b023478: 166, args=@0x7f940b023478: 166) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023478: 166, __args=@0x7f940b023478: 166) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023430) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 513 (Thread 0x7f9317bc8640 (LWP 2740171) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b544868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b544800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b544800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b544800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=165) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023410: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023420: 0x7f956f09f800, __args=@0x7f940b023428: 165) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023410: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023428: 165, __args=@0x7f940b023428: 165) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0233e8, args=@0x7f940b023428: 165, args=@0x7f940b023428: 165) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023428: 165, __args=@0x7f940b023428: 165) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0233e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 512 (Thread 0x7f93185c9640 (LWP 2740170) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b544568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b544500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b544500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b544500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=164) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0233c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0233d0: 0x7f956f09f800, __args=@0x7f940b0233d8: 164) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0233c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0233d8: 164, __args=@0x7f940b0233d8: 164) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023398, args=@0x7f940b0233d8: 164, args=@0x7f940b0233d8: 164) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0233d8: 164, __args=@0x7f940b0233d8: 164) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023390) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 511 (Thread 0x7f9318fca640 (LWP 2740168) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b544268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b544200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b544200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b544200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=163) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023370: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023380: 0x7f956f09f800, __args=@0x7f940b023388: 163) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023370: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023388: 163, __args=@0x7f940b023388: 163) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023348, args=@0x7f940b023388: 163, args=@0x7f940b023388: 163) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023388: 163, __args=@0x7f940b023388: 163) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 510 (Thread 0x7f93199cb640 (LWP 2740166) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b543f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=162) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023320: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023330: 0x7f956f09f800, __args=@0x7f940b023338: 162) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023320: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023338: 162, __args=@0x7f940b023338: 162) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0232f8, args=@0x7f940b023338: 162, args=@0x7f940b023338: 162) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023338: 162, __args=@0x7f940b023338: 162) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0232f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 509 (Thread 0x7f931a3cc640 (LWP 2740164) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b543c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=161) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0232d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0232e0: 0x7f956f09f800, __args=@0x7f940b0232e8: 161) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0232d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0232e8: 161, __args=@0x7f940b0232e8: 161) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0232a8, args=@0x7f940b0232e8: 161, args=@0x7f940b0232e8: 161) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0232e8: 161, __args=@0x7f940b0232e8: 161) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0232a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 508 (Thread 0x7f931adcd640 (LWP 2740163) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b543900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=160) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023280: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023290: 0x7f956f09f800, __args=@0x7f940b023298: 160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023280: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023298: 160, __args=@0x7f940b023298: 160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023258, args=@0x7f940b023298: 160, args=@0x7f940b023298: 160) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023298: 160, __args=@0x7f940b023298: 160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023250) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 507 (Thread 0x7f931b7ce640 (LWP 2740162) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b543600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=159) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023230: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023240: 0x7f956f09f800, __args=@0x7f940b023248: 159) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023230: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023248: 159, __args=@0x7f940b023248: 159) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023208, args=@0x7f940b023248: 159, args=@0x7f940b023248: 159) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023248: 159, __args=@0x7f940b023248: 159) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 506 (Thread 0x7f931c1cf640 (LWP 2740161) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b543300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=158) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0231e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0231f0: 0x7f956f09f800, __args=@0x7f940b0231f8: 158) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0231e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0231f8: 158, __args=@0x7f940b0231f8: 158) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0231b8, args=@0x7f940b0231f8: 158, args=@0x7f940b0231f8: 158) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0231f8: 158, __args=@0x7f940b0231f8: 158) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0231b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 505 (Thread 0x7f931cbd0640 (LWP 2740160) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b543000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=157) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023190: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0231a0: 0x7f956f09f800, __args=@0x7f940b0231a8: 157) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023190: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0231a8: 157, __args=@0x7f940b0231a8: 157) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023168, args=@0x7f940b0231a8: 157, args=@0x7f940b0231a8: 157) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0231a8: 157, __args=@0x7f940b0231a8: 157) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 504 (Thread 0x7f931d5d1640 (LWP 2740159) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c2d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c2d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c2d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c2d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=156) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023140: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023150: 0x7f956f09f800, __args=@0x7f940b023158: 156) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023140: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023158: 156, __args=@0x7f940b023158: 156) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023118, args=@0x7f940b023158: 156, args=@0x7f940b023158: 156) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023158: 156, __args=@0x7f940b023158: 156) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023110) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 503 (Thread 0x7f931dfd2640 (LWP 2740157) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c2a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c2a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c2a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c2a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=155) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0230f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023100: 0x7f956f09f800, __args=@0x7f940b023108: 155) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0230f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023108: 155, __args=@0x7f940b023108: 155) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0230c8, args=@0x7f940b023108: 155, args=@0x7f940b023108: 155) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023108: 155, __args=@0x7f940b023108: 155) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0230c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 502 (Thread 0x7f931e9d3640 (LWP 2740156) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c2768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c2700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c2700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c2700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=154) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0230a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0230b0: 0x7f956f09f800, __args=@0x7f940b0230b8: 154) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0230a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0230b8: 154, __args=@0x7f940b0230b8: 154) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023078, args=@0x7f940b0230b8: 154, args=@0x7f940b0230b8: 154) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0230b8: 154, __args=@0x7f940b0230b8: 154) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023070) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 501 (Thread 0x7f931f3d4640 (LWP 2740155) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c2468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c2400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c2400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c2400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=153) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023050: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023060: 0x7f956f09f800, __args=@0x7f940b023068: 153) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023050: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023068: 153, __args=@0x7f940b023068: 153) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023028, args=@0x7f940b023068: 153, args=@0x7f940b023068: 153) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023068: 153, __args=@0x7f940b023068: 153) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023020) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 500 (Thread 0x7f931fdd5640 (LWP 2740154) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c2168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c2100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c2100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c2100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=152) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023000: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023010: 0x7f956f09f800, __args=@0x7f940b023018: 152) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023000: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023018: 152, __args=@0x7f940b023018: 152) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022fd8, args=@0x7f940b023018: 152, args=@0x7f940b023018: 152) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023018: 152, __args=@0x7f940b023018: 152) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022fd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 499 (Thread 0x7f93207d6640 (LWP 2740153) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c1e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c1e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c1e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c1e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=151) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022fb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022fc0: 0x7f956f09f800, __args=@0x7f940b022fc8: 151) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022fb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022fc8: 151, __args=@0x7f940b022fc8: 151) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022f88, args=@0x7f940b022fc8: 151, args=@0x7f940b022fc8: 151) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022fc8: 151, __args=@0x7f940b022fc8: 151) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022f80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 498 (Thread 0x7f93211d7640 (LWP 2740152) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c1b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c1b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c1b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c1b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=150) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022f60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022f70: 0x7f956f09f800, __args=@0x7f940b022f78: 150) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022f60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022f78: 150, __args=@0x7f940b022f78: 150) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022f38, args=@0x7f940b022f78: 150, args=@0x7f940b022f78: 150) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022f78: 150, __args=@0x7f940b022f78: 150) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022f30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 497 (Thread 0x7f9321bd8640 (LWP 2740151) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c1868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c1800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c1800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c1800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=149) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022f10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022f20: 0x7f956f09f800, __args=@0x7f940b022f28: 149) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022f10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022f28: 149, __args=@0x7f940b022f28: 149) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022ee8, args=@0x7f940b022f28: 149, args=@0x7f940b022f28: 149) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022f28: 149, __args=@0x7f940b022f28: 149) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022ee0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 496 (Thread 0x7f93225d9640 (LWP 2740150) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c1568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c1500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c1500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c1500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=148) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ec0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022ed0: 0x7f956f09f800, __args=@0x7f940b022ed8: 148) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ec0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022ed8: 148, __args=@0x7f940b022ed8: 148) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022e98, args=@0x7f940b022ed8: 148, args=@0x7f940b022ed8: 148) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022ed8: 148, __args=@0x7f940b022ed8: 148) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022e90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 495 (Thread 0x7f9322fda640 (LWP 2740149) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c1268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c1200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c1200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c1200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=147) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022e70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022e80: 0x7f956f09f800, __args=@0x7f940b022e88: 147) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022e70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022e88: 147, __args=@0x7f940b022e88: 147) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022e48, args=@0x7f940b022e88: 147, args=@0x7f940b022e88: 147) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022e88: 147, __args=@0x7f940b022e88: 147) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022e40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 494 (Thread 0x7f93239db640 (LWP 2740148) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=146) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022e20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022e30: 0x7f956f09f800, __args=@0x7f940b022e38: 146) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022e20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022e38: 146, __args=@0x7f940b022e38: 146) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022df8, args=@0x7f940b022e38: 146, args=@0x7f940b022e38: 146) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022e38: 146, __args=@0x7f940b022e38: 146) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022df0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 493 (Thread 0x7f93243dc640 (LWP 2740147) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=145) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022dd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022de0: 0x7f956f09f800, __args=@0x7f940b022de8: 145) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022dd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022de8: 145, __args=@0x7f940b022de8: 145) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022da8, args=@0x7f940b022de8: 145, args=@0x7f940b022de8: 145) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022de8: 145, __args=@0x7f940b022de8: 145) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022da0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 492 (Thread 0x7f9324ddd640 (LWP 2740146) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=144) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022d80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022d90: 0x7f956f09f800, __args=@0x7f940b022d98: 144) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022d80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022d98: 144, __args=@0x7f940b022d98: 144) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022d58, args=@0x7f940b022d98: 144, args=@0x7f940b022d98: 144) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022d98: 144, __args=@0x7f940b022d98: 144) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022d50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 491 (Thread 0x7f93257de640 (LWP 2740145) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=143) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022d30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022d40: 0x7f956f09f800, __args=@0x7f940b022d48: 143) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022d30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022d48: 143, __args=@0x7f940b022d48: 143) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022d08, args=@0x7f940b022d48: 143, args=@0x7f940b022d48: 143) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022d48: 143, __args=@0x7f940b022d48: 143) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022d00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 490 (Thread 0x7f93261df640 (LWP 2740144) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=142) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ce0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022cf0: 0x7f956f09f800, __args=@0x7f940b022cf8: 142) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ce0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022cf8: 142, __args=@0x7f940b022cf8: 142) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022cb8, args=@0x7f940b022cf8: 142, args=@0x7f940b022cf8: 142) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022cf8: 142, __args=@0x7f940b022cf8: 142) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022cb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 489 (Thread 0x7f9326be0640 (LWP 2740143) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=141) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022c90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022ca0: 0x7f956f09f800, __args=@0x7f940b022ca8: 141) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022c90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022ca8: 141, __args=@0x7f940b022ca8: 141) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022c68, args=@0x7f940b022ca8: 141, args=@0x7f940b022ca8: 141) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022ca8: 141, __args=@0x7f940b022ca8: 141) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022c60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 488 (Thread 0x7f93275e1640 (LWP 2740142) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43fd68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43fd00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43fd00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43fd00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=140) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022c40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022c50: 0x7f956f09f800, __args=@0x7f940b022c58: 140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022c40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022c58: 140, __args=@0x7f940b022c58: 140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022c18, args=@0x7f940b022c58: 140, args=@0x7f940b022c58: 140) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022c58: 140, __args=@0x7f940b022c58: 140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022c10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 487 (Thread 0x7f93287e2640 (LWP 2740141) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43fa68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43fa00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43fa00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43fa00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=139) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022bf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022c00: 0x7f956f09f800, __args=@0x7f940b022c08: 139) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022bf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022c08: 139, __args=@0x7f940b022c08: 139) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022bc8, args=@0x7f940b022c08: 139, args=@0x7f940b022c08: 139) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022c08: 139, __args=@0x7f940b022c08: 139) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022bc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 486 (Thread 0x7f93291e3640 (LWP 2740140) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43f768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43f700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43f700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43f700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=138) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ba0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022bb0: 0x7f956f09f800, __args=@0x7f940b022bb8: 138) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ba0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022bb8: 138, __args=@0x7f940b022bb8: 138) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022b78, args=@0x7f940b022bb8: 138, args=@0x7f940b022bb8: 138) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022bb8: 138, __args=@0x7f940b022bb8: 138) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022b70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 485 (Thread 0x7f9329be4640 (LWP 2740139) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43f468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43f400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43f400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43f400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=137) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022b50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022b60: 0x7f956f09f800, __args=@0x7f940b022b68: 137) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022b50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022b68: 137, __args=@0x7f940b022b68: 137) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022b28, args=@0x7f940b022b68: 137, args=@0x7f940b022b68: 137) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022b68: 137, __args=@0x7f940b022b68: 137) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022b20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 484 (Thread 0x7f932a5e5640 (LWP 2740138) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43f168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43f100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43f100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43f100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=136) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022b00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022b10: 0x7f956f09f800, __args=@0x7f940b022b18: 136) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022b00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022b18: 136, __args=@0x7f940b022b18: 136) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022ad8, args=@0x7f940b022b18: 136, args=@0x7f940b022b18: 136) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022b18: 136, __args=@0x7f940b022b18: 136) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022ad0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 483 (Thread 0x7f932afe6640 (LWP 2740137) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43ee68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43ee00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43ee00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43ee00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=135) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ab0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022ac0: 0x7f956f09f800, __args=@0x7f940b022ac8: 135) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ab0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022ac8: 135, __args=@0x7f940b022ac8: 135) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022a88, args=@0x7f940b022ac8: 135, args=@0x7f940b022ac8: 135) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022ac8: 135, __args=@0x7f940b022ac8: 135) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 482 (Thread 0x7f932b9e7640 (LWP 2740136) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43eb68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43eb00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43eb00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43eb00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=134) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022a60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022a70: 0x7f956f09f800, __args=@0x7f940b022a78: 134) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022a60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022a78: 134, __args=@0x7f940b022a78: 134) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022a38, args=@0x7f940b022a78: 134, args=@0x7f940b022a78: 134) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022a78: 134, __args=@0x7f940b022a78: 134) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022a30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 481 (Thread 0x7f932c3e8640 (LWP 2740135) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43e868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43e800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43e800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43e800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=133) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022a10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022a20: 0x7f956f09f800, __args=@0x7f940b022a28: 133) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022a10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022a28: 133, __args=@0x7f940b022a28: 133) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0229e8, args=@0x7f940b022a28: 133, args=@0x7f940b022a28: 133) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022a28: 133, __args=@0x7f940b022a28: 133) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0229e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 480 (Thread 0x7f932cde9640 (LWP 2740134) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43e568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43e500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43e500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43e500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=132) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0229c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0229d0: 0x7f956f09f800, __args=@0x7f940b0229d8: 132) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0229c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0229d8: 132, __args=@0x7f940b0229d8: 132) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022998, args=@0x7f940b0229d8: 132, args=@0x7f940b0229d8: 132) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0229d8: 132, __args=@0x7f940b0229d8: 132) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022990) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 479 (Thread 0x7f932d7ea640 (LWP 2740133) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43e268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43e200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43e200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43e200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=131) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022970: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022980: 0x7f956f09f800, __args=@0x7f940b022988: 131) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022970: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022988: 131, __args=@0x7f940b022988: 131) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022948, args=@0x7f940b022988: 131, args=@0x7f940b022988: 131) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022988: 131, __args=@0x7f940b022988: 131) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 478 (Thread 0x7f932e1eb640 (LWP 2740132) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43df68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43df00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43df00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43df00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=130) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022920: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022930: 0x7f956f09f800, __args=@0x7f940b022938: 130) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022920: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022938: 130, __args=@0x7f940b022938: 130) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0228f8, args=@0x7f940b022938: 130, args=@0x7f940b022938: 130) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022938: 130, __args=@0x7f940b022938: 130) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0228f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 477 (Thread 0x7f932ebec640 (LWP 2740131) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43dc68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43dc00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43dc00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43dc00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=129) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0228d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0228e0: 0x7f956f09f800, __args=@0x7f940b0228e8: 129) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0228d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0228e8: 129, __args=@0x7f940b0228e8: 129) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0228a8, args=@0x7f940b0228e8: 129, args=@0x7f940b0228e8: 129) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0228e8: 129, __args=@0x7f940b0228e8: 129) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0228a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 476 (Thread 0x7f932f5ed640 (LWP 2740130) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43d968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43d900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43d900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43d900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=128) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022880: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022890: 0x7f956f09f800, __args=@0x7f940b022898: 128) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022880: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022898: 128, __args=@0x7f940b022898: 128) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022858, args=@0x7f940b022898: 128, args=@0x7f940b022898: 128) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022898: 128, __args=@0x7f940b022898: 128) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022850) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 475 (Thread 0x7f932ffee640 (LWP 2740129) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43d668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43d600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43d600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43d600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=127) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022830: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022840: 0x7f956f09f800, __args=@0x7f940b022848: 127) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022830: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022848: 127, __args=@0x7f940b022848: 127) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022808, args=@0x7f940b022848: 127, args=@0x7f940b022848: 127) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022848: 127, __args=@0x7f940b022848: 127) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022800) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 474 (Thread 0x7f93309ef640 (LWP 2740128) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43d368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43d300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43d300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43d300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=126) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0227e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0227f0: 0x7f956f09f800, __args=@0x7f940b0227f8: 126) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0227e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0227f8: 126, __args=@0x7f940b0227f8: 126) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0227b8, args=@0x7f940b0227f8: 126, args=@0x7f940b0227f8: 126) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0227f8: 126, __args=@0x7f940b0227f8: 126) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0227b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 473 (Thread 0x7f93313f0640 (LWP 2740127) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43d068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43d000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b43d000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43d000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=125) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022790: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0227a0: 0x7f956f09f800, __args=@0x7f940b0227a8: 125) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022790: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0227a8: 125, __args=@0x7f940b0227a8: 125) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022768, args=@0x7f940b0227a8: 125, args=@0x7f940b0227a8: 125) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0227a8: 125, __args=@0x7f940b0227a8: 125) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022760) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 472 (Thread 0x7f9331df1640 (LWP 2740126) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b9d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b9d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b9d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b9d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=124) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022740: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022750: 0x7f956f09f800, __args=@0x7f940b022758: 124) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022740: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022758: 124, __args=@0x7f940b022758: 124) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022718, args=@0x7f940b022758: 124, args=@0x7f940b022758: 124) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022758: 124, __args=@0x7f940b022758: 124) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022710) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 471 (Thread 0x7f93327f2640 (LWP 2740125) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b9a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b9a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b9a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b9a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=123) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0226f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022700: 0x7f956f09f800, __args=@0x7f940b022708: 123) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0226f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022708: 123, __args=@0x7f940b022708: 123) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0226c8, args=@0x7f940b022708: 123, args=@0x7f940b022708: 123) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022708: 123, __args=@0x7f940b022708: 123) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0226c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 470 (Thread 0x7f93331f3640 (LWP 2740124) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b9768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b9700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b9700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b9700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=122) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0226a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0226b0: 0x7f956f09f800, __args=@0x7f940b0226b8: 122) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0226a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0226b8: 122, __args=@0x7f940b0226b8: 122) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022678, args=@0x7f940b0226b8: 122, args=@0x7f940b0226b8: 122) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0226b8: 122, __args=@0x7f940b0226b8: 122) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022670) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 469 (Thread 0x7f9333bf4640 (LWP 2740123) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b9468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b9400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b9400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b9400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=121) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022650: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022660: 0x7f956f09f800, __args=@0x7f940b022668: 121) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022650: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022668: 121, __args=@0x7f940b022668: 121) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022628, args=@0x7f940b022668: 121, args=@0x7f940b022668: 121) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022668: 121, __args=@0x7f940b022668: 121) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022620) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 468 (Thread 0x7f93345f5640 (LWP 2740122) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b9168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b9100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b9100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b9100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=120) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022600: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022610: 0x7f956f09f800, __args=@0x7f940b022618: 120) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022600: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022618: 120, __args=@0x7f940b022618: 120) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0225d8, args=@0x7f940b022618: 120, args=@0x7f940b022618: 120) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022618: 120, __args=@0x7f940b022618: 120) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0225d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 467 (Thread 0x7f9334ff6640 (LWP 2740121) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b8e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b8e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b8e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b8e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=119) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0225b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0225c0: 0x7f956f09f800, __args=@0x7f940b0225c8: 119) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0225b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0225c8: 119, __args=@0x7f940b0225c8: 119) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022588, args=@0x7f940b0225c8: 119, args=@0x7f940b0225c8: 119) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0225c8: 119, __args=@0x7f940b0225c8: 119) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 466 (Thread 0x7f93359f7640 (LWP 2740120) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b8b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b8b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b8b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b8b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=118) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022560: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022570: 0x7f956f09f800, __args=@0x7f940b022578: 118) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022560: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022578: 118, __args=@0x7f940b022578: 118) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022538, args=@0x7f940b022578: 118, args=@0x7f940b022578: 118) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022578: 118, __args=@0x7f940b022578: 118) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022530) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 465 (Thread 0x7f9336bf8640 (LWP 2740119) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b8868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b8800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b8800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b8800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=117) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022510: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022520: 0x7f956f09f800, __args=@0x7f940b022528: 117) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022510: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022528: 117, __args=@0x7f940b022528: 117) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0224e8, args=@0x7f940b022528: 117, args=@0x7f940b022528: 117) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022528: 117, __args=@0x7f940b022528: 117) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0224e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 464 (Thread 0x7f93375f9640 (LWP 2740118) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b8568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b8500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b8500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b8500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=116) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0224c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0224d0: 0x7f956f09f800, __args=@0x7f940b0224d8: 116) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0224c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0224d8: 116, __args=@0x7f940b0224d8: 116) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022498, args=@0x7f940b0224d8: 116, args=@0x7f940b0224d8: 116) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0224d8: 116, __args=@0x7f940b0224d8: 116) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022490) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 463 (Thread 0x7f9337ffa640 (LWP 2740117) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b8268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b8200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b8200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b8200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=115) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022470: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022480: 0x7f956f09f800, __args=@0x7f940b022488: 115) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022470: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022488: 115, __args=@0x7f940b022488: 115) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022448, args=@0x7f940b022488: 115, args=@0x7f940b022488: 115) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022488: 115, __args=@0x7f940b022488: 115) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 462 (Thread 0x7f93389fb640 (LWP 2740116) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=114) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022420: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022430: 0x7f956f09f800, __args=@0x7f940b022438: 114) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022420: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022438: 114, __args=@0x7f940b022438: 114) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0223f8, args=@0x7f940b022438: 114, args=@0x7f940b022438: 114) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022438: 114, __args=@0x7f940b022438: 114) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0223f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 461 (Thread 0x7f93393fc640 (LWP 2740115) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=113) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0223d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0223e0: 0x7f956f09f800, __args=@0x7f940b0223e8: 113) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0223d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0223e8: 113, __args=@0x7f940b0223e8: 113) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0223a8, args=@0x7f940b0223e8: 113, args=@0x7f940b0223e8: 113) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0223e8: 113, __args=@0x7f940b0223e8: 113) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0223a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 460 (Thread 0x7f9339dfd640 (LWP 2740114) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=112) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022380: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022390: 0x7f956f09f800, __args=@0x7f940b022398: 112) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022380: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022398: 112, __args=@0x7f940b022398: 112) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022358, args=@0x7f940b022398: 112, args=@0x7f940b022398: 112) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022398: 112, __args=@0x7f940b022398: 112) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 459 (Thread 0x7f933a9fe640 (LWP 2740113) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=111) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022330: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022340: 0x7f956f09f800, __args=@0x7f940b022348: 111) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022330: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022348: 111, __args=@0x7f940b022348: 111) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022308, args=@0x7f940b022348: 111, args=@0x7f940b022348: 111) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022348: 111, __args=@0x7f940b022348: 111) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 458 (Thread 0x7f933b3ff640 (LWP 2740112) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=110) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0222e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0222f0: 0x7f956f09f800, __args=@0x7f940b0222f8: 110) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0222e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0222f8: 110, __args=@0x7f940b0222f8: 110) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0222b8, args=@0x7f940b0222f8: 110, args=@0x7f940b0222f8: 110) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0222f8: 110, __args=@0x7f940b0222f8: 110) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0222b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 457 (Thread 0x7f933c5ff640 (LWP 2740111) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=109) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022290: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0222a0: 0x7f956f09f800, __args=@0x7f940b0222a8: 109) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022290: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0222a8: 109, __args=@0x7f940b0222a8: 109) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022268, args=@0x7f940b0222a8: 109, args=@0x7f940b0222a8: 109) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0222a8: 109, __args=@0x7f940b0222a8: 109) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 456 (Thread 0x7f933d5ff640 (LWP 2740110) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b336d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b336d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b336d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b336d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=108) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022240: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022250: 0x7f956f09f800, __args=@0x7f940b022258: 108) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022240: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022258: 108, __args=@0x7f940b022258: 108) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022218, args=@0x7f940b022258: 108, args=@0x7f940b022258: 108) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022258: 108, __args=@0x7f940b022258: 108) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022210) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 455 (Thread 0x7f933e3ff640 (LWP 2740109) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b336a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b336a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b336a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b336a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=107) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0221f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022200: 0x7f956f09f800, __args=@0x7f940b022208: 107) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0221f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022208: 107, __args=@0x7f940b022208: 107) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0221c8, args=@0x7f940b022208: 107, args=@0x7f940b022208: 107) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022208: 107, __args=@0x7f940b022208: 107) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0221c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 454 (Thread 0x7f933f7fe640 (LWP 2740108) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b336768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b336700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b336700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b336700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=106) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0221a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0221b0: 0x7f956f09f800, __args=@0x7f940b0221b8: 106) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0221a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0221b8: 106, __args=@0x7f940b0221b8: 106) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022178, args=@0x7f940b0221b8: 106, args=@0x7f940b0221b8: 106) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0221b8: 106, __args=@0x7f940b0221b8: 106) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022170) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 453 (Thread 0x7f93403ff640 (LWP 2740107) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b336468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b336400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b336400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b336400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=105) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022150: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022160: 0x7f956f09f800, __args=@0x7f940b022168: 105) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022150: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022168: 105, __args=@0x7f940b022168: 105) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022128, args=@0x7f940b022168: 105, args=@0x7f940b022168: 105) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022168: 105, __args=@0x7f940b022168: 105) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022120) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 452 (Thread 0x7f93415ff640 (LWP 2740106) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b336168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b336100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b336100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b336100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=104) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022100: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022110: 0x7f956f09f800, __args=@0x7f940b022118: 104) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022100: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022118: 104, __args=@0x7f940b022118: 104) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0220d8, args=@0x7f940b022118: 104, args=@0x7f940b022118: 104) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022118: 104, __args=@0x7f940b022118: 104) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0220d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 451 (Thread 0x7f93425fe640 (LWP 2740105) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b335e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b335e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b335e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b335e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=103) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0220b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0220c0: 0x7f956f09f800, __args=@0x7f940b0220c8: 103) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0220b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0220c8: 103, __args=@0x7f940b0220c8: 103) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022088, args=@0x7f940b0220c8: 103, args=@0x7f940b0220c8: 103) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0220c8: 103, __args=@0x7f940b0220c8: 103) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 450 (Thread 0x7f9342fff640 (LWP 2740104) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b335b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b335b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b335b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b335b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=102) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022060: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022070: 0x7f956f09f800, __args=@0x7f940b022078: 102) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022060: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022078: 102, __args=@0x7f940b022078: 102) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022038, args=@0x7f940b022078: 102, args=@0x7f940b022078: 102) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022078: 102, __args=@0x7f940b022078: 102) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022030) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 449 (Thread 0x7f9343fff640 (LWP 2740103) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b335868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b335800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b335800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b335800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=101) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022010: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022020: 0x7f956f09f800, __args=@0x7f940b022028: 101) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022010: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022028: 101, __args=@0x7f940b022028: 101) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021fe8, args=@0x7f940b022028: 101, args=@0x7f940b022028: 101) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022028: 101, __args=@0x7f940b022028: 101) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021fe0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 448 (Thread 0x7f9344fff640 (LWP 2740102) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b335568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b335500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b335500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b335500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=100) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021fc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021fd0: 0x7f956f09f800, __args=@0x7f940b021fd8: 100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021fc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021fd8: 100, __args=@0x7f940b021fd8: 100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021f98, args=@0x7f940b021fd8: 100, args=@0x7f940b021fd8: 100) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021fd8: 100, __args=@0x7f940b021fd8: 100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021f90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 447 (Thread 0x7f9345fff640 (LWP 2740101) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b335268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b335200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b335200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b335200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=99) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021f70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021f80: 0x7f956f09f800, __args=@0x7f940b021f88: 99) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021f70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021f88: 99, __args=@0x7f940b021f88: 99) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021f48, args=@0x7f940b021f88: 99, args=@0x7f940b021f88: 99) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021f88: 99, __args=@0x7f940b021f88: 99) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021f40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 446 (Thread 0x7f9346dfe640 (LWP 2740100) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b334f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=98) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021f20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021f30: 0x7f956f09f800, __args=@0x7f940b021f38: 98) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021f20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021f38: 98, __args=@0x7f940b021f38: 98) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021ef8, args=@0x7f940b021f38: 98, args=@0x7f940b021f38: 98) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021f38: 98, __args=@0x7f940b021f38: 98) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021ef0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 445 (Thread 0x7f93479ff640 (LWP 2740099) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b334c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=97) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ed0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021ee0: 0x7f956f09f800, __args=@0x7f940b021ee8: 97) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ed0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021ee8: 97, __args=@0x7f940b021ee8: 97) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021ea8, args=@0x7f940b021ee8: 97, args=@0x7f940b021ee8: 97) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021ee8: 97, __args=@0x7f940b021ee8: 97) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021ea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 444 (Thread 0x7f9348dff640 (LWP 2740098) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b334900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=96) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021e80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021e90: 0x7f956f09f800, __args=@0x7f940b021e98: 96) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021e80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021e98: 96, __args=@0x7f940b021e98: 96) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021e58, args=@0x7f940b021e98: 96, args=@0x7f940b021e98: 96) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021e98: 96, __args=@0x7f940b021e98: 96) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021e50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 443 (Thread 0x7f9349cfd640 (LWP 2740097) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b334600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=95) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021e30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021e40: 0x7f956f09f800, __args=@0x7f940b021e48: 95) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021e30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021e48: 95, __args=@0x7f940b021e48: 95) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021e08, args=@0x7f940b021e48: 95, args=@0x7f940b021e48: 95) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021e48: 95, __args=@0x7f940b021e48: 95) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021e00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 442 (Thread 0x7f934a8fe640 (LWP 2740096) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b334300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=94) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021de0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021df0: 0x7f956f09f800, __args=@0x7f940b021df8: 94) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021de0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021df8: 94, __args=@0x7f940b021df8: 94) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021db8, args=@0x7f940b021df8: 94, args=@0x7f940b021df8: 94) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021df8: 94, __args=@0x7f940b021df8: 94) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021db0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 441 (Thread 0x7f934b2ff640 (LWP 2740095) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b334000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=93) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021d90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021da0: 0x7f956f09f800, __args=@0x7f940b021da8: 93) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021d90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021da8: 93, __args=@0x7f940b021da8: 93) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021d68, args=@0x7f940b021da8: 93, args=@0x7f940b021da8: 93) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021da8: 93, __args=@0x7f940b021da8: 93) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021d60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 440 (Thread 0x7f9425435640 (LWP 2740094) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bbee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bbe80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bbe80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bbe80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=92) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021d40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021d50: 0x7f956f09f800, __args=@0x7f940b021d58: 92) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021d40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021d58: 92, __args=@0x7f940b021d58: 92) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021d18, args=@0x7f940b021d58: 92, args=@0x7f940b021d58: 92) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021d58: 92, __args=@0x7f940b021d58: 92) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021d10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 439 (Thread 0x7f934d7ff640 (LWP 2740093) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bbbe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bbb80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bbb80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bbb80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=91) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021cf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021d00: 0x7f956f09f800, __args=@0x7f940b021d08: 91) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021cf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021d08: 91, __args=@0x7f940b021d08: 91) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021cc8, args=@0x7f940b021d08: 91, args=@0x7f940b021d08: 91) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021d08: 91, __args=@0x7f940b021d08: 91) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 438 (Thread 0x7f934e7ff640 (LWP 2740092) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bb8e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bb880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bb880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bb880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=90) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ca0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021cb0: 0x7f956f09f800, __args=@0x7f940b021cb8: 90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ca0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021cb8: 90, __args=@0x7f940b021cb8: 90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021c78, args=@0x7f940b021cb8: 90, args=@0x7f940b021cb8: 90) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021cb8: 90, __args=@0x7f940b021cb8: 90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021c70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 437 (Thread 0x7f934f7ff640 (LWP 2740091) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bb5e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bb580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bb580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bb580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=89) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021c50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021c60: 0x7f956f09f800, __args=@0x7f940b021c68: 89) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021c50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021c68: 89, __args=@0x7f940b021c68: 89) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021c28, args=@0x7f940b021c68: 89, args=@0x7f940b021c68: 89) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021c68: 89, __args=@0x7f940b021c68: 89) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021c20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 436 (Thread 0x7f9350bff640 (LWP 2740090) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bb2e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bb280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bb280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bb280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021c00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021c10: 0x7f956f09f800, __args=@0x7f940b021c18: 88) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021c00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021c18: 88, __args=@0x7f940b021c18: 88) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021bd8, args=@0x7f940b021c18: 88, args=@0x7f940b021c18: 88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021c18: 88, __args=@0x7f940b021c18: 88) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021bd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 435 (Thread 0x7f9351bff640 (LWP 2740089) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bafe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2baf80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2baf80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2baf80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=87) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021bb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021bc0: 0x7f956f09f800, __args=@0x7f940b021bc8: 87) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021bb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021bc8: 87, __args=@0x7f940b021bc8: 87) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021b88, args=@0x7f940b021bc8: 87, args=@0x7f940b021bc8: 87) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021bc8: 87, __args=@0x7f940b021bc8: 87) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021b80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 434 (Thread 0x7f9352bff640 (LWP 2740088) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bace8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bac80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bac80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bac80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=86) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021b60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021b70: 0x7f956f09f800, __args=@0x7f940b021b78: 86) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021b60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021b78: 86, __args=@0x7f940b021b78: 86) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021b38, args=@0x7f940b021b78: 86, args=@0x7f940b021b78: 86) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021b78: 86, __args=@0x7f940b021b78: 86) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021b30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 433 (Thread 0x7f93539ff640 (LWP 2740087) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2ba9e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2ba980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2ba980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2ba980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=85) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021b10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021b20: 0x7f956f09f800, __args=@0x7f940b021b28: 85) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021b10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021b28: 85, __args=@0x7f940b021b28: 85) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021ae8, args=@0x7f940b021b28: 85, args=@0x7f940b021b28: 85) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021b28: 85, __args=@0x7f940b021b28: 85) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021ae0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 432 (Thread 0x7f9354bff640 (LWP 2740086) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2ba6e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2ba680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2ba680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2ba680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=84) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ac0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021ad0: 0x7f956f09f800, __args=@0x7f940b021ad8: 84) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ac0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021ad8: 84, __args=@0x7f940b021ad8: 84) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021a98, args=@0x7f940b021ad8: 84, args=@0x7f940b021ad8: 84) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021ad8: 84, __args=@0x7f940b021ad8: 84) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021a90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 431 (Thread 0x7f9356bff640 (LWP 2740085) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2ba3e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2ba380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2ba380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2ba380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=83) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021a70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021a80: 0x7f956f09f800, __args=@0x7f940b021a88: 83) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021a70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021a88: 83, __args=@0x7f940b021a88: 83) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021a48, args=@0x7f940b021a88: 83, args=@0x7f940b021a88: 83) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021a88: 83, __args=@0x7f940b021a88: 83) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021a40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 430 (Thread 0x7f9357bff640 (LWP 2740084) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2ba0e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2ba080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2ba080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2ba080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=82) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021a20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021a30: 0x7f956f09f800, __args=@0x7f940b021a38: 82) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021a20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021a38: 82, __args=@0x7f940b021a38: 82) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0219f8, args=@0x7f940b021a38: 82, args=@0x7f940b021a38: 82) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021a38: 82, __args=@0x7f940b021a38: 82) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0219f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 429 (Thread 0x7f9358bff640 (LWP 2740083) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2b9de8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2b9d80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2b9d80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2b9d80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=81) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0219d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0219e0: 0x7f956f09f800, __args=@0x7f940b0219e8: 81) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0219d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0219e8: 81, __args=@0x7f940b0219e8: 81) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0219a8, args=@0x7f940b0219e8: 81, args=@0x7f940b0219e8: 81) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0219e8: 81, __args=@0x7f940b0219e8: 81) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0219a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 428 (Thread 0x7f93599ff640 (LWP 2740082) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2b9ae8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2b9a80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2b9a80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2b9a80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=80) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021980: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021990: 0x7f956f09f800, __args=@0x7f940b021998: 80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021980: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021998: 80, __args=@0x7f940b021998: 80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021958, args=@0x7f940b021998: 80, args=@0x7f940b021998: 80) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021998: 80, __args=@0x7f940b021998: 80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021950) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 427 (Thread 0x7f935b9fe640 (LWP 2740081) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2b97e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2b9780, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2b9780, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2b9780, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=79) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021930: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021940: 0x7f956f09f800, __args=@0x7f940b021948: 79) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021930: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021948: 79, __args=@0x7f940b021948: 79) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021908, args=@0x7f940b021948: 79, args=@0x7f940b021948: 79) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021948: 79, __args=@0x7f940b021948: 79) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 426 (Thread 0x7f935c5ff640 (LWP 2740080) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2b94e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2b9480, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2b9480, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2b9480, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=78) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0218e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0218f0: 0x7f956f09f800, __args=@0x7f940b0218f8: 78) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0218e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0218f8: 78, __args=@0x7f940b0218f8: 78) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0218b8, args=@0x7f940b0218f8: 78, args=@0x7f940b0218f8: 78) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0218f8: 78, __args=@0x7f940b0218f8: 78) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0218b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 425 (Thread 0x7f935d5ff640 (LWP 2740079) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2b91e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2b9180, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b2b9180, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2b9180, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=77) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021890: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0218a0: 0x7f956f09f800, __args=@0x7f940b0218a8: 77) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021890: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0218a8: 77, __args=@0x7f940b0218a8: 77) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021868, args=@0x7f940b0218a8: 77, args=@0x7f940b0218a8: 77) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0218a8: 77, __args=@0x7f940b0218a8: 77) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021860) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 424 (Thread 0x7f935e3ff640 (LWP 2740078) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b208ee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b208e80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b208e80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b208e80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=76) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021840: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021850: 0x7f956f09f800, __args=@0x7f940b021858: 76) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021840: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021858: 76, __args=@0x7f940b021858: 76) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021818, args=@0x7f940b021858: 76, args=@0x7f940b021858: 76) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021858: 76, __args=@0x7f940b021858: 76) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021810) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 423 (Thread 0x7f935fdff640 (LWP 2740077) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b208be8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b208b80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b208b80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b208b80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=75) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0217f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021800: 0x7f956f09f800, __args=@0x7f940b021808: 75) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0217f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021808: 75, __args=@0x7f940b021808: 75) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0217c8, args=@0x7f940b021808: 75, args=@0x7f940b021808: 75) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021808: 75, __args=@0x7f940b021808: 75) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0217c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 422 (Thread 0x7f9360dfe640 (LWP 2740076) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2088e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b208880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b208880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b208880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=74) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0217a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0217b0: 0x7f956f09f800, __args=@0x7f940b0217b8: 74) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0217a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0217b8: 74, __args=@0x7f940b0217b8: 74) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021778, args=@0x7f940b0217b8: 74, args=@0x7f940b0217b8: 74) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0217b8: 74, __args=@0x7f940b0217b8: 74) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021770) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 421 (Thread 0x7f93617ff640 (LWP 2740075) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2085e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b208580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b208580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b208580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=73) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021750: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021760: 0x7f956f09f800, __args=@0x7f940b021768: 73) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021750: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021768: 73, __args=@0x7f940b021768: 73) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021728, args=@0x7f940b021768: 73, args=@0x7f940b021768: 73) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021768: 73, __args=@0x7f940b021768: 73) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 420 (Thread 0x7f9362bff640 (LWP 2740074) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2082e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b208280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b208280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b208280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=72) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021700: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021710: 0x7f956f09f800, __args=@0x7f940b021718: 72) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021700: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021718: 72, __args=@0x7f940b021718: 72) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0216d8, args=@0x7f940b021718: 72, args=@0x7f940b021718: 72) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021718: 72, __args=@0x7f940b021718: 72) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0216d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 419 (Thread 0x7f93639fe640 (LWP 2740073) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b207fe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207f80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b207f80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207f80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=71) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0216b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0216c0: 0x7f956f09f800, __args=@0x7f940b0216c8: 71) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0216b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0216c8: 71, __args=@0x7f940b0216c8: 71) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021688, args=@0x7f940b0216c8: 71, args=@0x7f940b0216c8: 71) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0216c8: 71, __args=@0x7f940b0216c8: 71) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 418 (Thread 0x7f93643ff640 (LWP 2740072) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b207ce8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207c80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b207c80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207c80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=70) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021660: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021670: 0x7f956f09f800, __args=@0x7f940b021678: 70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021660: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021678: 70, __args=@0x7f940b021678: 70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021638, args=@0x7f940b021678: 70, args=@0x7f940b021678: 70) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021678: 70, __args=@0x7f940b021678: 70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021630) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 417 (Thread 0x7f9365bfa640 (LWP 2740071) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2079e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b207980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=69) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021610: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021620: 0x7f956f09f800, __args=@0x7f940b021628: 69) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021610: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021628: 69, __args=@0x7f940b021628: 69) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0215e8, args=@0x7f940b021628: 69, args=@0x7f940b021628: 69) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021628: 69, __args=@0x7f940b021628: 69) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0215e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 416 (Thread 0x7f93665fb640 (LWP 2740070) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af82ae08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af82ae00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 415 (Thread 0x7f9366ffc640 (LWP 2740069) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2076e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b207680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0215c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0215d0: 0x7f956f09f800, __args=@0x7f940b0215d8: 68) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0215c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0215d8: 68, __args=@0x7f940b0215d8: 68) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021598, args=@0x7f940b0215d8: 68, args=@0x7f940b0215d8: 68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0215d8: 68, __args=@0x7f940b0215d8: 68) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021590) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 414 (Thread 0x7f93685fc640 (LWP 2740068) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af82ada8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af82ada0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 413 (Thread 0x7f93691fd640 (LWP 2740067) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2073e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b207380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=67) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021570: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021580: 0x7f956f09f800, __args=@0x7f940b021588: 67) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021570: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021588: 67, __args=@0x7f940b021588: 67) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021548, args=@0x7f940b021588: 67, args=@0x7f940b021588: 67) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021588: 67, __args=@0x7f940b021588: 67) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 412 (Thread 0x7f9369bfe640 (LWP 2740066) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2070e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b207080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=66) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021520: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021530: 0x7f956f09f800, __args=@0x7f940b021538: 66) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021520: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021538: 66, __args=@0x7f940b021538: 66) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0214f8, args=@0x7f940b021538: 66, args=@0x7f940b021538: 66) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021538: 66, __args=@0x7f940b021538: 66) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0214f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 411 (Thread 0x7f936b5ff640 (LWP 2740065) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b206de8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b206d80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b206d80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b206d80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=65) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0214d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0214e0: 0x7f956f09f800, __args=@0x7f940b0214e8: 65) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0214d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0214e8: 65, __args=@0x7f940b0214e8: 65) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0214a8, args=@0x7f940b0214e8: 65, args=@0x7f940b0214e8: 65) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0214e8: 65, __args=@0x7f940b0214e8: 65) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0214a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 410 (Thread 0x7f936abfe640 (LWP 2740064) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af82ad48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af82ad40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 409 (Thread 0x7f936cbfe640 (LWP 2740063) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b206ae8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b206a80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b206a80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b206a80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=64) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021480: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021490: 0x7f956f09f800, __args=@0x7f940b021498: 64) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021480: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021498: 64, __args=@0x7f940b021498: 64) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021458, args=@0x7f940b021498: 64, args=@0x7f940b021498: 64) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021498: 64, __args=@0x7f940b021498: 64) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021450) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 408 (Thread 0x7f936d9ff640 (LWP 2740062) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2067e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b206780, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b206780, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b206780, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=63) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021430: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021440: 0x7f956f09f800, __args=@0x7f940b021448: 63) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021430: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021448: 63, __args=@0x7f940b021448: 63) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021408, args=@0x7f940b021448: 63, args=@0x7f940b021448: 63) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021448: 63, __args=@0x7f940b021448: 63) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 407 (Thread 0x7f936e9ff640 (LWP 2740061) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2064e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b206480, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b206480, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b206480, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=62) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0213e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0213f0: 0x7f956f09f800, __args=@0x7f940b0213f8: 62) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0213e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0213f8: 62, __args=@0x7f940b0213f8: 62) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0213b8, args=@0x7f940b0213f8: 62, args=@0x7f940b0213f8: 62) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0213f8: 62, __args=@0x7f940b0213f8: 62) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0213b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 406 (Thread 0x7f936f7fe640 (LWP 2740060) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af82ace8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af82ace0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 405 (Thread 0x7f93705ff640 (LWP 2740059) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2061e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b206180, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b206180, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b206180, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=61) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021390: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0213a0: 0x7f956f09f800, __args=@0x7f940b0213a8: 61) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021390: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0213a8: 61, __args=@0x7f940b0213a8: 61) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021368, args=@0x7f940b0213a8: 61, args=@0x7f940b0213a8: 61) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0213a8: 61, __args=@0x7f940b0213a8: 61) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 404 (Thread 0x7f93713fd640 (LWP 2740058) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af82ac88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af82ac80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 403 (Thread 0x7f9371ffe640 (LWP 2740057) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b185ee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b185e80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b185e80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b185e80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=60) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021340: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021350: 0x7f956f09f800, __args=@0x7f940b021358: 60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021340: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021358: 60, __args=@0x7f940b021358: 60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021318, args=@0x7f940b021358: 60, args=@0x7f940b021358: 60) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021358: 60, __args=@0x7f940b021358: 60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021310) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 402 (Thread 0x7f9372ffe640 (LWP 2740056) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b185be8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b185b80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b185b80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b185b80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=59) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0212f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021300: 0x7f956f09f800, __args=@0x7f940b021308: 59) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0212f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021308: 59, __args=@0x7f940b021308: 59) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0212c8, args=@0x7f940b021308: 59, args=@0x7f940b021308: 59) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021308: 59, __args=@0x7f940b021308: 59) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0212c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 401 (Thread 0x7f93739ff640 (LWP 2740055) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1858e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b185880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b185880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b185880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=58) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0212a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0212b0: 0x7f956f09f800, __args=@0x7f940b0212b8: 58) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0212a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0212b8: 58, __args=@0x7f940b0212b8: 58) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021278, args=@0x7f940b0212b8: 58, args=@0x7f940b0212b8: 58) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0212b8: 58, __args=@0x7f940b0212b8: 58) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021270) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 400 (Thread 0x7f93749ff640 (LWP 2740054) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1855e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b185580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b185580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b185580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=57) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021250: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021260: 0x7f956f09f800, __args=@0x7f940b021268: 57) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021250: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021268: 57, __args=@0x7f940b021268: 57) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021228, args=@0x7f940b021268: 57, args=@0x7f940b021268: 57) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021268: 57, __args=@0x7f940b021268: 57) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 399 (Thread 0x7f93759ff640 (LWP 2740053) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1852e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b185280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b185280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b185280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=56) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021200: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021210: 0x7f956f09f800, __args=@0x7f940b021218: 56) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021200: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021218: 56, __args=@0x7f940b021218: 56) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0211d8, args=@0x7f940b021218: 56, args=@0x7f940b021218: 56) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021218: 56, __args=@0x7f940b021218: 56) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0211d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 398 (Thread 0x7f93769ff640 (LWP 2740052) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b184fe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184f80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b184f80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184f80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=55) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0211b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0211c0: 0x7f956f09f800, __args=@0x7f940b0211c8: 55) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0211b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0211c8: 55, __args=@0x7f940b0211c8: 55) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021188, args=@0x7f940b0211c8: 55, args=@0x7f940b0211c8: 55) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0211c8: 55, __args=@0x7f940b0211c8: 55) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 397 (Thread 0x7f93777fe640 (LWP 2740051) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b184ce8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184c80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b184c80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184c80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=54) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021160: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021170: 0x7f956f09f800, __args=@0x7f940b021178: 54) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021160: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021178: 54, __args=@0x7f940b021178: 54) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021138, args=@0x7f940b021178: 54, args=@0x7f940b021178: 54) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021178: 54, __args=@0x7f940b021178: 54) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021130) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 396 (Thread 0x7f93783ff640 (LWP 2740050) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1849e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b184980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=53) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021110: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021120: 0x7f956f09f800, __args=@0x7f940b021128: 53) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021110: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021128: 53, __args=@0x7f940b021128: 53) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0210e8, args=@0x7f940b021128: 53, args=@0x7f940b021128: 53) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021128: 53, __args=@0x7f940b021128: 53) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0210e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 395 (Thread 0x7f93793ff640 (LWP 2740049) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1846e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b184680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=52) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0210c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0210d0: 0x7f956f09f800, __args=@0x7f940b0210d8: 52) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0210c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0210d8: 52, __args=@0x7f940b0210d8: 52) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021098, args=@0x7f940b0210d8: 52, args=@0x7f940b0210d8: 52) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0210d8: 52, __args=@0x7f940b0210d8: 52) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021090) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 394 (Thread 0x7f937a3ff640 (LWP 2740048) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1843e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b184380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=51) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021070: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021080: 0x7f956f09f800, __args=@0x7f940b021088: 51) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021070: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021088: 51, __args=@0x7f940b021088: 51) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021048, args=@0x7f940b021088: 51, args=@0x7f940b021088: 51) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021088: 51, __args=@0x7f940b021088: 51) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 393 (Thread 0x7f937b7ff640 (LWP 2740047) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1840e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b184080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=50) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021020: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021030: 0x7f956f09f800, __args=@0x7f940b021038: 50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021020: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021038: 50, __args=@0x7f940b021038: 50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020ff8, args=@0x7f940b021038: 50, args=@0x7f940b021038: 50) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021038: 50, __args=@0x7f940b021038: 50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020ff0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 392 (Thread 0x7f937c5ff640 (LWP 2740046) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b183de8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b183d80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b183d80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b183d80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=49) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020fd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020fe0: 0x7f956f09f800, __args=@0x7f940b020fe8: 49) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020fd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020fe8: 49, __args=@0x7f940b020fe8: 49) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020fa8, args=@0x7f940b020fe8: 49, args=@0x7f940b020fe8: 49) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020fe8: 49, __args=@0x7f940b020fe8: 49) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020fa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 391 (Thread 0x7f937d5ff640 (LWP 2740045) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b183ae8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b183a80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b183a80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b183a80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020f80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020f90: 0x7f956f09f800, __args=@0x7f940b020f98: 48) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020f80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020f98: 48, __args=@0x7f940b020f98: 48) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020f58, args=@0x7f940b020f98: 48, args=@0x7f940b020f98: 48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020f98: 48, __args=@0x7f940b020f98: 48) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020f50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 390 (Thread 0x7f937e3fe640 (LWP 2740044) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1837e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b183780, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b183780, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b183780, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=47) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020f30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020f40: 0x7f956f09f800, __args=@0x7f940b020f48: 47) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020f30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020f48: 47, __args=@0x7f940b020f48: 47) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020f08, args=@0x7f940b020f48: 47, args=@0x7f940b020f48: 47) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020f48: 47, __args=@0x7f940b020f48: 47) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020f00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 389 (Thread 0x7f937f1ff640 (LWP 2740043) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1834e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b183480, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b183480, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b183480, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=46) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020ee0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020ef0: 0x7f956f09f800, __args=@0x7f940b020ef8: 46) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020ee0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020ef8: 46, __args=@0x7f940b020ef8: 46) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020eb8, args=@0x7f940b020ef8: 46, args=@0x7f940b020ef8: 46) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020ef8: 46, __args=@0x7f940b020ef8: 46) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020eb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 388 (Thread 0x7f93801ff640 (LWP 2740042) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1831e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b183180, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b183180, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b183180, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=45) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020e90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020ea0: 0x7f956f09f800, __args=@0x7f940b020ea8: 45) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020e90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020ea8: 45, __args=@0x7f940b020ea8: 45) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020e68, args=@0x7f940b020ea8: 45, args=@0x7f940b020ea8: 45) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020ea8: 45, __args=@0x7f940b020ea8: 45) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020e60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 387 (Thread 0x7f93811ff640 (LWP 2740041) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b102ee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b102e80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b102e80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b102e80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=44) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020e40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020e50: 0x7f956f09f800, __args=@0x7f940b020e58: 44) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020e40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020e58: 44, __args=@0x7f940b020e58: 44) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020e18, args=@0x7f940b020e58: 44, args=@0x7f940b020e58: 44) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020e58: 44, __args=@0x7f940b020e58: 44) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020e10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 386 (Thread 0x7f93825ff640 (LWP 2740040) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b102be8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b102b80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b102b80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b102b80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=43) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020df0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020e00: 0x7f956f09f800, __args=@0x7f940b020e08: 43) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020df0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020e08: 43, __args=@0x7f940b020e08: 43) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020dc8, args=@0x7f940b020e08: 43, args=@0x7f940b020e08: 43) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020e08: 43, __args=@0x7f940b020e08: 43) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020dc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 385 (Thread 0x7f93835ff640 (LWP 2740039) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1028e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b102880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b102880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b102880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=42) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020da0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020db0: 0x7f956f09f800, __args=@0x7f940b020db8: 42) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020da0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020db8: 42, __args=@0x7f940b020db8: 42) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020d78, args=@0x7f940b020db8: 42, args=@0x7f940b020db8: 42) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020db8: 42, __args=@0x7f940b020db8: 42) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020d70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 384 (Thread 0x7f93843ff640 (LWP 2740038) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1025e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b102580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b102580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b102580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=41) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020d50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020d60: 0x7f956f09f800, __args=@0x7f940b020d68: 41) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020d50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020d68: 41, __args=@0x7f940b020d68: 41) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020d28, args=@0x7f940b020d68: 41, args=@0x7f940b020d68: 41) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020d68: 41, __args=@0x7f940b020d68: 41) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020d20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 383 (Thread 0x7f9385dff640 (LWP 2740037) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1022e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b102280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b102280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b102280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=40) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020d00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020d10: 0x7f956f09f800, __args=@0x7f940b020d18: 40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020d00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020d18: 40, __args=@0x7f940b020d18: 40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020cd8, args=@0x7f940b020d18: 40, args=@0x7f940b020d18: 40) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020d18: 40, __args=@0x7f940b020d18: 40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020cd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 382 (Thread 0x7f9386dff640 (LWP 2740036) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b101fe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101f80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b101f80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101f80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=39) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020cb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020cc0: 0x7f956f09f800, __args=@0x7f940b020cc8: 39) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020cb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020cc8: 39, __args=@0x7f940b020cc8: 39) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020c88, args=@0x7f940b020cc8: 39, args=@0x7f940b020cc8: 39) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020cc8: 39, __args=@0x7f940b020cc8: 39) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020c80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 381 (Thread 0x7f9387dff640 (LWP 2740035) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b101ce8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101c80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b101c80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101c80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=38) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020c60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020c70: 0x7f956f09f800, __args=@0x7f940b020c78: 38) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020c60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020c78: 38, __args=@0x7f940b020c78: 38) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020c38, args=@0x7f940b020c78: 38, args=@0x7f940b020c78: 38) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020c78: 38, __args=@0x7f940b020c78: 38) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020c30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 380 (Thread 0x7f9388dff640 (LWP 2740034) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1019e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b101980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=37) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020c10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020c20: 0x7f956f09f800, __args=@0x7f940b020c28: 37) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020c10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020c28: 37, __args=@0x7f940b020c28: 37) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020be8, args=@0x7f940b020c28: 37, args=@0x7f940b020c28: 37) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020c28: 37, __args=@0x7f940b020c28: 37) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020be0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 379 (Thread 0x7f9389dff640 (LWP 2740033) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1016e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b101680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=36) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020bc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020bd0: 0x7f956f09f800, __args=@0x7f940b020bd8: 36) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020bc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020bd8: 36, __args=@0x7f940b020bd8: 36) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020b98, args=@0x7f940b020bd8: 36, args=@0x7f940b020bd8: 36) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020bd8: 36, __args=@0x7f940b020bd8: 36) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020b90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 378 (Thread 0x7f938adff640 (LWP 2740032) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1013e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b101380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=35) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020b70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020b80: 0x7f956f09f800, __args=@0x7f940b020b88: 35) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020b70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020b88: 35, __args=@0x7f940b020b88: 35) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020b48, args=@0x7f940b020b88: 35, args=@0x7f940b020b88: 35) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020b88: 35, __args=@0x7f940b020b88: 35) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 377 (Thread 0x7f938bcff640 (LWP 2740031) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1010e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b101080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=34) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020b20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020b30: 0x7f956f09f800, __args=@0x7f940b020b38: 34) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020b20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020b38: 34, __args=@0x7f940b020b38: 34) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020af8, args=@0x7f940b020b38: 34, args=@0x7f940b020b38: 34) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020b38: 34, __args=@0x7f940b020b38: 34) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020af0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 376 (Thread 0x7f938d3ff640 (LWP 2740030) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b100de8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b100d80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b100d80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b100d80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=33) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020ad0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020ae0: 0x7f956f09f800, __args=@0x7f940b020ae8: 33) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020ad0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020ae8: 33, __args=@0x7f940b020ae8: 33) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020aa8, args=@0x7f940b020ae8: 33, args=@0x7f940b020ae8: 33) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020ae8: 33, __args=@0x7f940b020ae8: 33) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020aa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 375 (Thread 0x7f938e3ff640 (LWP 2740029) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b100ae8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b100a80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b100a80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b100a80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=32) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020a80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020a90: 0x7f956f09f800, __args=@0x7f940b020a98: 32) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020a80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020a98: 32, __args=@0x7f940b020a98: 32) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020a58, args=@0x7f940b020a98: 32, args=@0x7f940b020a98: 32) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020a98: 32, __args=@0x7f940b020a98: 32) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020a50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 374 (Thread 0x7f938f1ff640 (LWP 2740028) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1007e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b100780, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b100780, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b100780, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=31) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020a30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020a40: 0x7f956f09f800, __args=@0x7f940b020a48: 31) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020a30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020a48: 31, __args=@0x7f940b020a48: 31) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020a08, args=@0x7f940b020a48: 31, args=@0x7f940b020a48: 31) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020a48: 31, __args=@0x7f940b020a48: 31) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 373 (Thread 0x7f93905ff640 (LWP 2740027) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1004e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b100480, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b100480, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b100480, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=30) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0209e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0209f0: 0x7f956f09f800, __args=@0x7f940b0209f8: 30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0209e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0209f8: 30, __args=@0x7f940b0209f8: 30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0209b8, args=@0x7f940b0209f8: 30, args=@0x7f940b0209f8: 30) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0209f8: 30, __args=@0x7f940b0209f8: 30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0209b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 372 (Thread 0x7f93915ff640 (LWP 2740026) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1001e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b100180, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b100180, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b100180, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=29) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020990: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0209a0: 0x7f956f09f800, __args=@0x7f940b0209a8: 29) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020990: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0209a8: 29, __args=@0x7f940b0209a8: 29) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020968, args=@0x7f940b0209a8: 29, args=@0x7f940b0209a8: 29) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0209a8: 29, __args=@0x7f940b0209a8: 29) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020960) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 371 (Thread 0x7f93923ff640 (LWP 2740025) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b077ee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b077e80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b077e80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b077e80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020940: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020950: 0x7f956f09f800, __args=@0x7f940b020958: 28) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020940: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020958: 28, __args=@0x7f940b020958: 28) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020918, args=@0x7f940b020958: 28, args=@0x7f940b020958: 28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020958: 28, __args=@0x7f940b020958: 28) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020910) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 370 (Thread 0x7f93935ff640 (LWP 2740024) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b077be8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b077b80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b077b80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b077b80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=27) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0208f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020900: 0x7f956f09f800, __args=@0x7f940b020908: 27) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0208f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020908: 27, __args=@0x7f940b020908: 27) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0208c8, args=@0x7f940b020908: 27, args=@0x7f940b020908: 27) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020908: 27, __args=@0x7f940b020908: 27) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0208c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 369 (Thread 0x7f93945ff640 (LWP 2740023) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0778e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b077880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b077880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b077880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=26) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0208a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0208b0: 0x7f956f09f800, __args=@0x7f940b0208b8: 26) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0208a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0208b8: 26, __args=@0x7f940b0208b8: 26) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020878, args=@0x7f940b0208b8: 26, args=@0x7f940b0208b8: 26) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0208b8: 26, __args=@0x7f940b0208b8: 26) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020870) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 368 (Thread 0x7f93955ff640 (LWP 2740022) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0775e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b077580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b077580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b077580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=25) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020850: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020860: 0x7f956f09f800, __args=@0x7f940b020868: 25) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020850: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020868: 25, __args=@0x7f940b020868: 25) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020828, args=@0x7f940b020868: 25, args=@0x7f940b020868: 25) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020868: 25, __args=@0x7f940b020868: 25) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020820) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 367 (Thread 0x7f93965ff640 (LWP 2740021) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0772e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b077280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b077280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b077280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=24) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020800: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020810: 0x7f956f09f800, __args=@0x7f940b020818: 24) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020800: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020818: 24, __args=@0x7f940b020818: 24) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0207d8, args=@0x7f940b020818: 24, args=@0x7f940b020818: 24) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020818: 24, __args=@0x7f940b020818: 24) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0207d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 366 (Thread 0x7f93973ff640 (LWP 2740020) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b076fe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076f80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b076f80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076f80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=23) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0207b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0207c0: 0x7f956f09f800, __args=@0x7f940b0207c8: 23) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0207b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0207c8: 23, __args=@0x7f940b0207c8: 23) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020788, args=@0x7f940b0207c8: 23, args=@0x7f940b0207c8: 23) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0207c8: 23, __args=@0x7f940b0207c8: 23) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 365 (Thread 0x7f93985ff640 (LWP 2740019) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b076ce8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076c80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b076c80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076c80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=22) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020760: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020770: 0x7f956f09f800, __args=@0x7f940b020778: 22) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020760: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020778: 22, __args=@0x7f940b020778: 22) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020738, args=@0x7f940b020778: 22, args=@0x7f940b020778: 22) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020778: 22, __args=@0x7f940b020778: 22) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020730) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 364 (Thread 0x7f93995ff640 (LWP 2740018) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0769e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b076980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=21) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020710: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020720: 0x7f956f09f800, __args=@0x7f940b020728: 21) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020710: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020728: 21, __args=@0x7f940b020728: 21) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0206e8, args=@0x7f940b020728: 21, args=@0x7f940b020728: 21) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020728: 21, __args=@0x7f940b020728: 21) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0206e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 363 (Thread 0x7f939a5ff640 (LWP 2740017) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0766e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b076680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=20) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0206c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0206d0: 0x7f956f09f800, __args=@0x7f940b0206d8: 20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0206c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0206d8: 20, __args=@0x7f940b0206d8: 20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020698, args=@0x7f940b0206d8: 20, args=@0x7f940b0206d8: 20) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0206d8: 20, __args=@0x7f940b0206d8: 20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020690) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 362 (Thread 0x7f939b4ff640 (LWP 2740016) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0763e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b076380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=19) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020670: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020680: 0x7f956f09f800, __args=@0x7f940b020688: 19) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020670: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020688: 19, __args=@0x7f940b020688: 19) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020648, args=@0x7f940b020688: 19, args=@0x7f940b020688: 19) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020688: 19, __args=@0x7f940b020688: 19) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 361 (Thread 0x7f939cbff640 (LWP 2740015) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0760e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b076080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=18) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020620: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020630: 0x7f956f09f800, __args=@0x7f940b020638: 18) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020620: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020638: 18, __args=@0x7f940b020638: 18) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0205f8, args=@0x7f940b020638: 18, args=@0x7f940b020638: 18) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020638: 18, __args=@0x7f940b020638: 18) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0205f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 360 (Thread 0x7f939d9ff640 (LWP 2740014) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b075de8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b075d80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b075d80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b075d80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=17) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0205d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0205e0: 0x7f956f09f800, __args=@0x7f940b0205e8: 17) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0205d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0205e8: 17, __args=@0x7f940b0205e8: 17) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0205a8, args=@0x7f940b0205e8: 17, args=@0x7f940b0205e8: 17) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0205e8: 17, __args=@0x7f940b0205e8: 17) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0205a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 359 (Thread 0x7f939ebff640 (LWP 2740013) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b075ae8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b075a80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b075a80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b075a80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=16) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020580: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020590: 0x7f956f09f800, __args=@0x7f940b020598: 16) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020580: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020598: 16, __args=@0x7f940b020598: 16) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020558, args=@0x7f940b020598: 16, args=@0x7f940b020598: 16) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020598: 16, __args=@0x7f940b020598: 16) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020550) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 358 (Thread 0x7f939fbff640 (LWP 2740012) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0757e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b075780, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b075780, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b075780, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=15) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020530: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020540: 0x7f956f09f800, __args=@0x7f940b020548: 15) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020530: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020548: 15, __args=@0x7f940b020548: 15) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020508, args=@0x7f940b020548: 15, args=@0x7f940b020548: 15) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020548: 15, __args=@0x7f940b020548: 15) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 357 (Thread 0x7f93a0ffe640 (LWP 2740011) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0754e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b075480, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b075480, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b075480, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=14) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0204e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0204f0: 0x7f956f09f800, __args=@0x7f940b0204f8: 14) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0204e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0204f8: 14, __args=@0x7f940b0204f8: 14) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0204b8, args=@0x7f940b0204f8: 14, args=@0x7f940b0204f8: 14) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0204f8: 14, __args=@0x7f940b0204f8: 14) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0204b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 356 (Thread 0x7f93a19ff640 (LWP 2740010) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0751e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b075180, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f940b075180, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b075180, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=13) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020490: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0204a0: 0x7f956f09f800, __args=@0x7f940b0204a8: 13) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020490: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0204a8: 13, __args=@0x7f940b0204a8: 13) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020468, args=@0x7f940b0204a8: 13, args=@0x7f940b0204a8: 13) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0204a8: 13, __args=@0x7f940b0204a8: 13) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020460) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 355 (Thread 0x7f93a2dff640 (LWP 2740009) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f9493790ee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f9493790e80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f9493790e80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f9493790e80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=12) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020440: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020450: 0x7f956f09f800, __args=@0x7f940b020458: 12) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020440: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020458: 12, __args=@0x7f940b020458: 12) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020418, args=@0x7f940b020458: 12, args=@0x7f940b020458: 12) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020458: 12, __args=@0x7f940b020458: 12) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020410) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 354 (Thread 0x7f93a3bfe640 (LWP 2740008) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f9493790be8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f9493790b80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f9493790b80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f9493790b80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=11) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0203f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020400: 0x7f956f09f800, __args=@0x7f940b020408: 11) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0203f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020408: 11, __args=@0x7f940b020408: 11) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0203c8, args=@0x7f940b020408: 11, args=@0x7f940b020408: 11) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020408: 11, __args=@0x7f940b020408: 11) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0203c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 353 (Thread 0x7f93a49ff640 (LWP 2740007) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f94937908e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f9493790880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f9493790880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f9493790880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=10) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0203a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0203b0: 0x7f956f09f800, __args=@0x7f940b0203b8: 10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0203a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0203b8: 10, __args=@0x7f940b0203b8: 10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020378, args=@0x7f940b0203b8: 10, args=@0x7f940b0203b8: 10) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0203b8: 10, __args=@0x7f940b0203b8: 10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020370) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 352 (Thread 0x7f93a5bff640 (LWP 2740006) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f94937905e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f9493790580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f9493790580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f9493790580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=9) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020350: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020360: 0x7f956f09f800, __args=@0x7f940b020368: 9) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020350: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020368: 9, __args=@0x7f940b020368: 9) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020328, args=@0x7f940b020368: 9, args=@0x7f940b020368: 9) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020368: 9, __args=@0x7f940b020368: 9) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 351 (Thread 0x7f93a69fe640 (LWP 2740005) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f94937902e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f9493790280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f9493790280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f9493790280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020300: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020310: 0x7f956f09f800, __args=@0x7f940b020318: 8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020300: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020318: 8, __args=@0x7f940b020318: 8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0202d8, args=@0x7f940b020318: 8, args=@0x7f940b020318: 8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020318: 8, __args=@0x7f940b020318: 8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0202d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 350 (Thread 0x7f93a75ff640 (LWP 2740004) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c25949d41 in std::__1::condition_variable::wait(std::__1::unique_lock&, DB::PipelineExecutorContext::wait()::$_0) (this=0x7f93a3ca24c0, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c25949ba2 in DB::PipelineExecutorContext::wait (this=0x7f93a3ca2440) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Executor/PipelineExecutorContext.cpp:114 +#5 0x0000558c259417d4 in DB::PipelineExecutor::wait (this=0x7f93a3ca2400) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Executor/PipelineExecutor.cpp:81 +#6 0x0000558c259419c2 in DB::PipelineExecutor::execute (this=0x7f93a3ca2400, result_handler=...) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Executor/PipelineExecutor.cpp:108 +#7 0x0000558c2595ee87 in DB::QueryExecutor::execute (this=0x7f93a3ca2400) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Executor/QueryExecutor.cpp:29 +#8 0x0000558c258716fc in DB::MPPTask::runImpl (this=0x7f9392664800) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Mpp/MPPTask.cpp:617 +#9 0x0000558c258768dd in DB::MPPTask::run()::$_0::operator()() const (this=0x7f939265ab28) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Mpp/MPPTask.cpp:234 +#10 0x0000558c258768b5 in std::__1::__invoke[abi:ue170006](DB::MPPTask::run()::$_0&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#11 0x0000558c25876875 in std::__1::__invoke_void_return_wrapper::__call[abi:ue170006](DB::MPPTask::run()::$_0&) (__args=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:415 +#12 0x0000558c2587684d in std::__1::__function::__alloc_func, void ()>::operator()[abi:ue170006]() (this=0x7f939265ab28) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:192 +#13 0x0000558c25875b49 in std::__1::__function::__func, void ()>::operator()() (this=0x7f939265ab20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:363 +#14 0x0000558c1abb48d2 in std::__1::__function::__value_func::operator()[abi:ue170006]() const (this=0x7f939265ab20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:517 +#15 0x0000558c1aba7925 in std::__1::function::operator()() const (this=0x7f939265ab20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:1168 +#16 0x0000558c238334d5 in std::__1::__invoke[abi:ue170006]>(std::__1::function&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#17 0x0000558c238334b9 in std::__1::__apply_tuple_impl[abi:ue170006], std::__1::tuple<>>(std::__1::function&&, std::__1::tuple<>&&, std::__1::__tuple_indices<>) (__f=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/tuple:1825 +#18 0x0000558c2383348d in std::__1::apply[abi:ue170006], std::__1::tuple<> >(std::__1::function&&, std::__1::tuple<>&&) (__f=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/tuple:1834 +#19 0x0000558c2383343b in DB::wrapInvocable>(bool, std::__1::function&&)::{lambda()#1}::operator()() (this=0x7f939265ab10) at /DATA/disk3/xzx/tiflash/dbms/src/Common/wrapInvocable.h:36 +#20 0x0000558c23833e09 in DB::ExecutableTask>(bool, std::__1::function&&)::{lambda()#1}>::execute() (this=0x7f939265ab00) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ExecutableTask.h:50 +#21 0x0000558c1ac03568 in DB::DynamicThreadPool::executeTask (task=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:124 +#22 0x0000558c1ac031ea in DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=7) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:138 +#23 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0202b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0202c0: 0x7f956f09f800, __args=@0x7f940b0202c8: 7) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#24 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0202b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0202c8: 7, __args=@0x7f940b0202c8: 7) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#25 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020288, args=@0x7f940b0202c8: 7, args=@0x7f940b0202c8: 7) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#26 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0202c8: 7, __args=@0x7f940b0202c8: 7) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#27 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#28 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 349 (Thread 0x7f93a85ff640 (LWP 2740003) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378fce8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378fc80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f949378fc80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378fc80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=6) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020260: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020270: 0x7f956f09f800, __args=@0x7f940b020278: 6) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020260: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020278: 6, __args=@0x7f940b020278: 6) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020238, args=@0x7f940b020278: 6, args=@0x7f940b020278: 6) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020278: 6, __args=@0x7f940b020278: 6) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020230) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 348 (Thread 0x7f93a95ff640 (LWP 2740002) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378f9e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378f980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f949378f980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378f980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=5) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020210: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020220: 0x7f956f09f800, __args=@0x7f940b020228: 5) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020210: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020228: 5, __args=@0x7f940b020228: 5) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0201e8, args=@0x7f940b020228: 5, args=@0x7f940b020228: 5) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020228: 5, __args=@0x7f940b020228: 5) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0201e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 347 (Thread 0x7f93aa6fc640 (LWP 2740001) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378f6e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378f680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f949378f680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378f680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=4) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0201c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0201d0: 0x7f956f09f800, __args=@0x7f940b0201d8: 4) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0201c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0201d8: 4, __args=@0x7f940b0201d8: 4) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020198, args=@0x7f940b0201d8: 4, args=@0x7f940b0201d8: 4) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0201d8: 4, __args=@0x7f940b0201d8: 4) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020190) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 346 (Thread 0x7f93ab0fd640 (LWP 2740000) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378f3e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378f380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f949378f380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378f380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=3) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020170: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020180: 0x7f956f09f800, __args=@0x7f940b020188: 3) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020170: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020188: 3, __args=@0x7f940b020188: 3) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020148, args=@0x7f940b020188: 3, args=@0x7f940b020188: 3) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020188: 3, __args=@0x7f940b020188: 3) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 345 (Thread 0x7f93ac5fd640 (LWP 2739999) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378f0e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378f080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f949378f080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378f080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=2) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020120: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020130: 0x7f956f09f800, __args=@0x7f940b020138: 2) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020120: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020138: 2, __args=@0x7f940b020138: 2) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0200f8, args=@0x7f940b020138: 2, args=@0x7f940b020138: 2) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020138: 2, __args=@0x7f940b020138: 2) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0200f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 344 (Thread 0x7f93acffe640 (LWP 2739998) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378ede8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378ed80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f949378ed80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378ed80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=1) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0200d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0200e0: 0x7f956f09f800, __args=@0x7f940b0200e8: 1) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0200d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0200e8: 1, __args=@0x7f940b0200e8: 1) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0200a8, args=@0x7f940b0200e8: 1, args=@0x7f940b0200e8: 1) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0200e8: 1, __args=@0x7f940b0200e8: 1) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0200a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 343 (Thread 0x7f93ad9ff640 (LWP 2739997) "FixedThread"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378ec68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 +#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378ec00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 +#5 DB::MPMCQueue > >::popObj (this=0x7f949378ec00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 +#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378ec00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 +#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 +#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020080: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020090: 0x7f956f09f800, __args=@0x7f940b020098: 0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020080: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020098: 0, __args=@0x7f940b020098: 0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020058, args=@0x7f940b020098: 0, args=@0x7f940b020098: 0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020098: 0, __args=@0x7f940b020098: 0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020050) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 342 (Thread 0x7f93babfc640 (LWP 2739996) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b2c3fca8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b2c3fca0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 341 (Thread 0x7f93bbffd640 (LWP 2739995) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b2c3fc48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b2c3fc40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 340 (Thread 0x7f93bc9fe640 (LWP 2739994) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b2c3fbe8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b2c3fbe0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 339 (Thread 0x7f93bd5ff640 (LWP 2739993) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b2c3fb88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b2c3fb80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 338 (Thread 0x7f93c2dff640 (LWP 2739992) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4426688) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4426680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 337 (Thread 0x7f93c3dff640 (LWP 2739991) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4426628) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4426620) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 336 (Thread 0x7f93c4bfe640 (LWP 2739990) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b44265c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b44265c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 335 (Thread 0x7f93c55ff640 (LWP 2739989) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4426568) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4426560) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 334 (Thread 0x7f93c68ff640 (LWP 2739988) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af435fc8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af435fc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 333 (Thread 0x7f93c7c7f640 (LWP 2739987) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af435f68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af435f60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 332 (Thread 0x7f93c93ff640 (LWP 2739986) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af435f08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af435f00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 331 (Thread 0x7f93ca1fe640 (LWP 2739985) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af435ea8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af435ea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 330 (Thread 0x7f93cb67f640 (LWP 2739984) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af435e48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af435e40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 329 (Thread 0x7f93d1dfe640 (LWP 2739983) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ac4207a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ac4207a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 328 (Thread 0x7f93d31ff640 (LWP 2739982) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ac420748) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ac420740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 327 (Thread 0x7f93d63ff640 (LWP 2739981) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00e28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00e20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 326 (Thread 0x7f93d73ff640 (LWP 2739980) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00dc8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00dc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 325 (Thread 0x7f93d83ff640 (LWP 2739979) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00d68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00d60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 324 (Thread 0x7f93d93fd640 (LWP 2739978) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa04ba8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa04ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 323 (Thread 0x7f93d9dfe640 (LWP 2739977) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3b068) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3b060) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 322 (Thread 0x7f93da7ff640 (LWP 2739976) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3b088) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3b080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 321 (Thread 0x7f93db7ff640 (LWP 2739975) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9a04ac8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9a04ac0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 320 (Thread 0x7f93dc7ff640 (LWP 2739974) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9a04a68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9a04a60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 319 (Thread 0x7f93ddfff640 (LWP 2739973) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9401368) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9401360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 318 (Thread 0x7f93defff640 (LWP 2739972) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00d08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00d00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 317 (Thread 0x7f93dffff640 (LWP 2739971) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b3a00648) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b3a00640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 316 (Thread 0x7f93e0dff640 (LWP 2739970) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3bd68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3bd60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 315 (Thread 0x7f93e21ff640 (LWP 2739969) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3bdc8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3bdc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 314 (Thread 0x7f93e2fff640 (LWP 2739968) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9a04a08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9a04a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 313 (Thread 0x7f93e3fff640 (LWP 2739967) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94c4801888) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94c4801880) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 312 (Thread 0x7f93e4dfe640 (LWP 2739966) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94c4801848) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94c4801840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 311 (Thread 0x7f93e5bff640 (LWP 2739965) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4a009a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4a009a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 310 (Thread 0x7f93e6bff640 (LWP 2739964) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b34019a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b34019a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 309 (Thread 0x7f93e7dff640 (LWP 2739963) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b7aebf68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b7aebf60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 308 (Thread 0x7f93e8d7e640 (LWP 2739962) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4a00928) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4a00920) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 307 (Thread 0x7f93e977f640 (LWP 2739961) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9a049a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9a049a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 306 (Thread 0x7f93eabff640 (LWP 2739960) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4c01508) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4c01500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 305 (Thread 0x7f93ebbff640 (LWP 2739959) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00ca8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00ca0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 304 (Thread 0x7f93ecbff640 (LWP 2739958) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00c48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 303 (Thread 0x7f93edefe640 (LWP 2739957) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ca0016c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ca0016c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 302 (Thread 0x7f93eebff640 (LWP 2739956) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3be08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3be00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 301 (Thread 0x7f93efdff640 (LWP 2739955) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3be88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3be80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 300 (Thread 0x7f93f0fff640 (LWP 2739954) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3bea8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3bea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 299 (Thread 0x7f93f1fff640 (LWP 2739953) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b3a005e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b3a005e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 298 (Thread 0x7f93f2fff640 (LWP 2739952) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94cdc012a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94cdc012a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 297 (Thread 0x7f93f3fff640 (LWP 2739951) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b0e00468) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b0e00460) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 296 (Thread 0x7f93f4fff640 (LWP 2739950) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94adc007c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94adc007c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 295 (Thread 0x7f93f5fff640 (LWP 2739949) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b0000588) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b0000580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 294 (Thread 0x7f93f6fff640 (LWP 2739948) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213f08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213f00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 293 (Thread 0x7f93f7fff640 (LWP 2739947) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b42005e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b42005e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 292 (Thread 0x7f93f8efe640 (LWP 2739946) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213ea8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213ea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 291 (Thread 0x7f93f98ff640 (LWP 2739945) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b1600728) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b1600720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 290 (Thread 0x7f93fabff640 (LWP 2739944) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213e48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213e40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 289 (Thread 0x7f93fbb7f640 (LWP 2739943) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bfe01368) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bfe01360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 288 (Thread 0x7f93fcbff640 (LWP 2739942) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213de8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213de0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 287 (Thread 0x7f93fe1f8640 (LWP 2739941) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bb3a3e28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bb3a3e20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 286 (Thread 0x7f93fedf9640 (LWP 2739940) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213d88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213d80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 285 (Thread 0x7f93ff7fa640 (LWP 2739939) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bba04448) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bba04440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 284 (Thread 0x7f94001fb640 (LWP 2739938) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213d28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213d20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 283 (Thread 0x7f9400bfc640 (LWP 2739937) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94cc001508) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94cc001500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 282 (Thread 0x7f94015fd640 (LWP 2739936) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213cc8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 281 (Thread 0x7f9401ffe640 (LWP 2739935) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94cae01248) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94cae01240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 280 (Thread 0x7f94029ff640 (LWP 2739934) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213c68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213c60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 279 (Thread 0x7f9403400640 (LWP 2739933) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94cdc01248) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94cdc01240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 278 (Thread 0x7f9403e01640 (LWP 2739932) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213c08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213c00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 277 (Thread 0x7f9404802640 (LWP 2739931) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9250168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9250160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 276 (Thread 0x7f9405203640 (LWP 2739930) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213ba8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 275 (Thread 0x7f9405c04640 (LWP 2739929) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ca001188) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ca001180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 274 (Thread 0x7f9406605640 (LWP 2739928) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213b48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 273 (Thread 0x7f9407006640 (LWP 2739927) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ab8014a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ab8014a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 272 (Thread 0x7f9407a07640 (LWP 2739926) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213ae8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213ae0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 271 (Thread 0x7f9408408640 (LWP 2739925) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ac200268) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ac200260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 270 (Thread 0x7f9408e09640 (LWP 2739924) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213a88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 269 (Thread 0x7f940980a640 (LWP 2739923) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94adc004c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94adc004c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 268 (Thread 0x7f940a20b640 (LWP 2739922) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213a28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213a20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 267 (Thread 0x7f940ac0c640 (LWP 2739921) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4a001a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4a001a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 266 (Thread 0x7f940c40d640 (LWP 2739920) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62139c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62139c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 265 (Thread 0x7f940ce0e640 (LWP 2739919) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af8013e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af8013e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 264 (Thread 0x7f940d80f640 (LWP 2739918) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213968) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213960) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 263 (Thread 0x7f940e210640 (LWP 2739917) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b00001e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b00001e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 262 (Thread 0x7f940ec11640 (LWP 2739916) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213908) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 261 (Thread 0x7f940f612640 (LWP 2739915) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b1e001a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b1e001a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 260 (Thread 0x7f9410013640 (LWP 2739914) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62138a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62138a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 259 (Thread 0x7f9410a14640 (LWP 2739913) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b16004a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b16004a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 258 (Thread 0x7f9411415640 (LWP 2739912) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213848) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 257 (Thread 0x7f9411e16640 (LWP 2739911) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b42002e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b42002e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 256 (Thread 0x7f9412817640 (LWP 2739910) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62137e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62137e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 255 (Thread 0x7f9413218640 (LWP 2739909) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b2c011e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b2c011e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 254 (Thread 0x7f9413c19640 (LWP 2739908) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213788) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 253 (Thread 0x7f941461a640 (LWP 2739907) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b3401228) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b3401220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 252 (Thread 0x7f941501b640 (LWP 2739906) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213728) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 251 (Thread 0x7f9415a1c640 (LWP 2739905) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b8a4e3c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b8a4e3c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 250 (Thread 0x7f941641d640 (LWP 2739904) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62136c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62136c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 249 (Thread 0x7f9416e1e640 (LWP 2739903) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b8a26768) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b8a26760) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 248 (Thread 0x7f941781f640 (LWP 2739902) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213668) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213660) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 247 (Thread 0x7f9418220640 (LWP 2739901) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b52001a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b52001a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 246 (Thread 0x7f9418c21640 (LWP 2739900) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213608) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 245 (Thread 0x7f9419622640 (LWP 2739899) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ba204488) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ba204480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 244 (Thread 0x7f941a023640 (LWP 2739898) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62135a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62135a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 243 (Thread 0x7f941aa24640 (LWP 2739897) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bb3e7528) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bb3e7520) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 242 (Thread 0x7f941b425640 (LWP 2739896) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213548) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 241 (Thread 0x7f941be26640 (LWP 2739895) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bba04068) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bba04060) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 240 (Thread 0x7f941c827640 (LWP 2739894) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62134e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62134e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 239 (Thread 0x7f941d228640 (LWP 2739893) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b7aeb508) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b7aeb500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 238 (Thread 0x7f941dc29640 (LWP 2739892) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213488) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 237 (Thread 0x7f941e62a640 (LWP 2739891) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b8a51c48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b8a51c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 236 (Thread 0x7f941f02b640 (LWP 2739890) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213428) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213420) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 235 (Thread 0x7f941fa2c640 (LWP 2739889) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b7404488) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b7404480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 234 (Thread 0x7f942042d640 (LWP 2739888) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62133c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62133c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 233 (Thread 0x7f9420e2e640 (LWP 2739887) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b92505e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b92505e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 232 (Thread 0x7f942182f640 (LWP 2739886) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213368) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 231 (Thread 0x7f9422230640 (LWP 2739885) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213308) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 230 (Thread 0x7f9422c31640 (LWP 2739884) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62132a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62132a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 229 (Thread 0x7f9423632640 (LWP 2739883) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213248) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 228 (Thread 0x7f9424033640 (LWP 2739882) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62131e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62131e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 227 (Thread 0x7f9424a34640 (LWP 2739881) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213188) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 226 (Thread 0x7f9428636640 (LWP 2739831) "grpc_global_tim"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x0000558c288f9ed9 in absl::lts_20211102::synchronization_internal::FutexImpl::WaitUntil (v=0x7f958456e740, val=0, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/futex.h:97 +#2 0x0000558c288f9bfb in absl::lts_20211102::synchronization_internal::Waiter::Wait (this=0x7f958456e740, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/waiter.cc:95 +#3 0x0000558c288f9816 in AbslInternalPerThreadSemWait_lts_20211102 (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.cc:93 +#4 0x0000558c288f6bbd in absl::lts_20211102::synchronization_internal::PerThreadSem::Wait (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.h:112 +#5 0x0000558c288efd56 in absl::lts_20211102::Mutex::DecrementSynchSem (mu=0x558c29567628 , w=0x7f958456e700, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:579 +#6 0x0000558c288f4e7a in absl::lts_20211102::CondVar::WaitCommon (this=0x558c29567630 , mutex=0x558c29567628 , t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2559 +#7 0x0000558c288f5058 in absl::lts_20211102::CondVar::WaitWithDeadline (this=0x558c29567630 , mu=0x558c29567628 , deadline=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2590 +#8 0x0000558c28330047 in gpr_cv_wait (cv=0x558c29567630 , mu=0x558c29567628 , abs_deadline=...) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gpr/sync_abseil.cc:92 +#9 0x0000558c27e19621 in wait_until (next=57318793) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:200 +#10 0x0000558c27e191e9 in timer_main_loop () at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:255 +#11 0x0000558c27e190ef in timer_thread (completed_thread_ptr=0x7f94b6bff000) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:284 +#12 0x0000558c28339f22 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::operator()(void*) const (this=0x7f9428602c87, v=0x7f94b6bff060) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:140 +#13 0x0000558c28339e09 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) (v=0x7f94b6bff060) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:110 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 225 (Thread 0x7f9429037640 (LWP 2739830) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bba04108) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bba04100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 224 (Thread 0x7f9429a38640 (LWP 2739829) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9a04168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9a04160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 223 (Thread 0x7f942a439640 (LWP 2739828) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ba204168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ba204160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 222 (Thread 0x7f942ae3a640 (LWP 2739827) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa04168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa04160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 221 (Thread 0x7f942b83b640 (LWP 2739826) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bb204168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bb204160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 220 (Thread 0x7f942c23c640 (LWP 2739825) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94937237e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94937237e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 219 (Thread 0x7f942cc3d640 (LWP 2739824) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f9493723788) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f9493723780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 218 (Thread 0x7f942d63e640 (LWP 2739823) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f9493723708) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f9493723700) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 217 (Thread 0x7f942e03f640 (LWP 2739822) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94937236a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94937236a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 216 (Thread 0x7f942ea40640 (LWP 2739821) "ThreadPool"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 +#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f9493723648) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 +#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f9493723640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 215 (Thread 0x7f942f441640 (LWP 2739820) "segment-sched"): +#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 +#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 +#2 0x00007f95847adc8b in std::__1::this_thread::sleep_for(std::__1::chrono::duration > const&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ab783b2 in std::__1::this_thread::sleep_for[abi:ue170006] >(std::__1::chrono::duration > const&) (__d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/this_thread.h:55 +#4 0x0000558c2331b5e0 in DB::DM::SegmentReadTaskScheduler::schedLoop (this=0x558c28f73ba0 ) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.cpp:324 +#5 0x0000558c23326ac9 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReadTaskScheduler::*&&)(), DB::DM::SegmentReadTaskScheduler*&&) (__f=@0x7f94937235e8: (void (DB::DM::SegmentReadTaskScheduler::*)(class DB::DM::SegmentReadTaskScheduler * const)) 0x558c2331b580 , __a0=@0x7f94937235f8: 0x558c28f73ba0 ) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#6 0x0000558c23326a4e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReadTaskScheduler::*)(), DB::DM::SegmentReadTaskScheduler*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReadTaskScheduler::*)(), DB::DM::SegmentReadTaskScheduler*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#7 0x0000558c23326822 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReadTaskScheduler::*)(), DB::DM::SegmentReadTaskScheduler*> >(void*) (__vp=0x7f94937235e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#8 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#9 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 214 (Thread 0x7f942fe42640 (LWP 2739819) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493721140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493721140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717cc8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717cd8: 0x7f9493721140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 213 (Thread 0x7f9430843640 (LWP 2739818) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94937210c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94937210c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717ca8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717cb8: 0x7f94937210c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717ca0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 212 (Thread 0x7f9431244640 (LWP 2739817) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493721040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493721040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717c88: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717c98: 0x7f9493721040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717c80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 211 (Thread 0x7f9431c45640 (LWP 2739816) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f94fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f94fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717c68: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717c78: 0x7f9493f94fc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717c60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 210 (Thread 0x7f9432646640 (LWP 2739815) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f94f40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f94f40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717c48: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717c58: 0x7f9493f94f40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 209 (Thread 0x7f9433047640 (LWP 2739814) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f94ec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f94ec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717c28: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717c38: 0x7f9493f94ec0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717c20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 208 (Thread 0x7f9433a48640 (LWP 2739813) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f94e40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f94e40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717c08: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717c18: 0x7f9493f94e40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717c00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 207 (Thread 0x7f9434449640 (LWP 2739812) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f7a900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f7a900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717be8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717bf8: 0x7f9493f7a900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717be0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 206 (Thread 0x7f9434e4a640 (LWP 2739811) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f6dd40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f6dd40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717bc8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717bd8: 0x7f9493f6dd40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717bc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 205 (Thread 0x7f943584b640 (LWP 2739810) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f6dcc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f6dcc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717ba8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717bb8: 0x7f9493f6dcc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 204 (Thread 0x7f943624c640 (LWP 2739809) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f6da00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f6da00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717b88: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717b98: 0x7f9493f6da00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717b80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 203 (Thread 0x7f9436c4d640 (LWP 2739808) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f6d680) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f6d680) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717b68: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717b78: 0x7f9493f6d680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717b60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 202 (Thread 0x7f943764e640 (LWP 2739807) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f6d600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f6d600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717b48: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717b58: 0x7f9493f6d600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 201 (Thread 0x7f943804f640 (LWP 2739806) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f60f80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f60f80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717b28: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717b38: 0x7f9493f60f80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717b20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 200 (Thread 0x7f9438a50640 (LWP 2739805) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f606c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f606c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717b08: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717b18: 0x7f9493f606c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717b00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 199 (Thread 0x7f9439451640 (LWP 2739804) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f60640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f60640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717ae8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717af8: 0x7f9493f60640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717ae0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 198 (Thread 0x7f9439e52640 (LWP 2739803) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f3dd40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f3dd40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717ac8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717ad8: 0x7f9493f3dd40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717ac0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 197 (Thread 0x7f943a853640 (LWP 2739802) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f3d600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f3d600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717aa8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717ab8: 0x7f9493f3d600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717aa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 196 (Thread 0x7f943b254640 (LWP 2739801) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f3d580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f3d580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717a88: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717a98: 0x7f9493f3d580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 195 (Thread 0x7f943bc55640 (LWP 2739800) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f3d440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f3d440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717a68: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717a78: 0x7f9493f3d440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717a60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 194 (Thread 0x7f943c656640 (LWP 2739799) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f2d1c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f2d1c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717a48: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717a58: 0x7f9493f2d1c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717a40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 193 (Thread 0x7f943d057640 (LWP 2739798) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f2d140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f2d140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717a28: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717a38: 0x7f9493f2d140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717a20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 192 (Thread 0x7f943da58640 (LWP 2739797) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f20400) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f20400) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717a08: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717a18: 0x7f9493f20400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 191 (Thread 0x7f943e459640 (LWP 2739796) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06d00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06d00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937179e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937179f8: 0x7f9493f06d00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937179e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 190 (Thread 0x7f943ee5a640 (LWP 2739795) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06c80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06c80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937179c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937179d8: 0x7f9493f06c80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937179c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 189 (Thread 0x7f943f85b640 (LWP 2739794) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06a80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06a80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937179a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937179b8: 0x7f9493f06a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937179a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 188 (Thread 0x7f944025c640 (LWP 2739793) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f067c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f067c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717988: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717998: 0x7f9493f067c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717980) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 187 (Thread 0x7f9440c5d640 (LWP 2739792) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717968: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717978: 0x7f9493f06740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717960) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 186 (Thread 0x7f944165e640 (LWP 2739791) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717948: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717958: 0x7f9493f06540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 185 (Thread 0x7f944205f640 (LWP 2739790) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717928: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717938: 0x7f9493f06340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717920) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 184 (Thread 0x7f9442a60640 (LWP 2739789) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f062c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f062c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717908: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717918: 0x7f9493f062c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 183 (Thread 0x7f9443461640 (LWP 2739788) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ef9c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ef9c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937178e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937178f8: 0x7f9493ef9c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937178e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 182 (Thread 0x7f9443e62640 (LWP 2739787) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ef9800) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ef9800) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937178c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937178d8: 0x7f9493ef9800) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937178c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 181 (Thread 0x7f9444863640 (LWP 2739786) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ef9780) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ef9780) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937178a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937178b8: 0x7f9493ef9780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937178a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 180 (Thread 0x7f9445264640 (LWP 2739785) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ef9280) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ef9280) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717888: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717898: 0x7f9493ef9280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717880) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 179 (Thread 0x7f9445c65640 (LWP 2739784) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee9cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee9cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717868: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717878: 0x7f9493ee9cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717860) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 178 (Thread 0x7f9446666640 (LWP 2739783) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee9c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee9c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717848: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717858: 0x7f9493ee9c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 177 (Thread 0x7f9447067640 (LWP 2739782) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee98c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee98c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717828: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717838: 0x7f9493ee98c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717820) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 176 (Thread 0x7f9447a68640 (LWP 2739781) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee96c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee96c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717808: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717818: 0x7f9493ee96c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717800) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 175 (Thread 0x7f9448469640 (LWP 2739780) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee9640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee9640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937177e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937177f8: 0x7f9493ee9640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937177e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 174 (Thread 0x7f9448e6a640 (LWP 2739779) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee9380) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee9380) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937177c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937177d8: 0x7f9493ee9380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937177c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 173 (Thread 0x7f944986b640 (LWP 2739778) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed9a00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed9a00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937177a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937177b8: 0x7f9493ed9a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937177a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 172 (Thread 0x7f944a26c640 (LWP 2739777) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed9980) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed9980) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717788: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717798: 0x7f9493ed9980) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 171 (Thread 0x7f944ac6d640 (LWP 2739776) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed9840) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed9840) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717768: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717778: 0x7f9493ed9840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717760) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 170 (Thread 0x7f944b66e640 (LWP 2739775) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed97c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed97c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717748: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717758: 0x7f9493ed97c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 169 (Thread 0x7f944c06f640 (LWP 2739774) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed9740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed9740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717728: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717738: 0x7f9493ed9740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 168 (Thread 0x7f944ca70640 (LWP 2739773) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed93c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed93c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717708: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717718: 0x7f9493ed93c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717700) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 167 (Thread 0x7f944d471640 (LWP 2739772) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed9040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed9040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937176e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937176f8: 0x7f9493ed9040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937176e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 166 (Thread 0x7f944de72640 (LWP 2739771) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ecffc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ecffc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937176c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937176d8: 0x7f9493ecffc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937176c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 165 (Thread 0x7f944e873640 (LWP 2739770) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebfb00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebfb00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937176a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937176b8: 0x7f9493ebfb00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937176a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 164 (Thread 0x7f944f274640 (LWP 2739769) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebf9c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebf9c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717688: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717698: 0x7f9493ebf9c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 163 (Thread 0x7f944fc75640 (LWP 2739768) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebf940) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebf940) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717668: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717678: 0x7f9493ebf940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717660) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 162 (Thread 0x7f9450676640 (LWP 2739767) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebf740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebf740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717648: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717658: 0x7f9493ebf740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 161 (Thread 0x7f9451077640 (LWP 2739766) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebf600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebf600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717628: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717638: 0x7f9493ebf600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717620) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 160 (Thread 0x7f9451a78640 (LWP 2739765) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebf580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebf580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717608: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717618: 0x7f9493ebf580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 159 (Thread 0x7f9452479640 (LWP 2739764) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493eb2c00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493eb2c00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937175e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937175f8: 0x7f9493eb2c00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937175e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 158 (Thread 0x7f9452e7a640 (LWP 2739763) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493eb2340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493eb2340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937175c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937175d8: 0x7f9493eb2340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937175c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 157 (Thread 0x7f945387b640 (LWP 2739762) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493eb22c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493eb22c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937175a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937175b8: 0x7f9493eb22c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937175a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 156 (Thread 0x7f945427c640 (LWP 2739761) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2f40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2f40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717588: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717598: 0x7f9493ea2f40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 155 (Thread 0x7f9454c7d640 (LWP 2739760) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2b00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2b00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717568: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717578: 0x7f9493ea2b00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717560) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 154 (Thread 0x7f945567e640 (LWP 2739759) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2a80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2a80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717548: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717558: 0x7f9493ea2a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 153 (Thread 0x7f945607f640 (LWP 2739758) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717528: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717538: 0x7f9493ea2640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717520) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 152 (Thread 0x7f9456a80640 (LWP 2739757) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea20c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea20c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717508: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717518: 0x7f9493ea20c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 151 (Thread 0x7f9457481640 (LWP 2739756) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2400) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2400) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937174e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937174f8: 0x7f9493ea2400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937174e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 150 (Thread 0x7f9457e82640 (LWP 2739755) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea22c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea22c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937174c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937174d8: 0x7f9493ea22c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937174c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 149 (Thread 0x7f9458883640 (LWP 2739754) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2240) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2240) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937174a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937174b8: 0x7f9493ea2240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937174a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 148 (Thread 0x7f9459284640 (LWP 2739753) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2100) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2100) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717448: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717458: 0x7f9493ea2100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 147 (Thread 0x7f9459c85640 (LWP 2739752) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e95e00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e95e00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717488: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717498: 0x7f9493e95e00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 146 (Thread 0x7f945a686640 (LWP 2739751) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e95cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e95cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717468: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717478: 0x7f9493e95cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717460) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 145 (Thread 0x7f945b087640 (LWP 2739750) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e95c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e95c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717428: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717438: 0x7f9493e95c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717420) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 144 (Thread 0x7f945ba88640 (LWP 2739749) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e95200) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e95200) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717408: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717418: 0x7f9493e95200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 143 (Thread 0x7f945c489640 (LWP 2739748) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e950c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e950c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937173e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937173f8: 0x7f9493e950c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937173e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 142 (Thread 0x7f945ce8a640 (LWP 2739747) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88e80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88e80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4ac8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4ad8: 0x7f9493e88e80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4ac0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 141 (Thread 0x7f945d88b640 (LWP 2739746) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88e00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88e00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4aa8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4ab8: 0x7f9493e88e00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4aa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 140 (Thread 0x7f945e28c640 (LWP 2739745) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4a88: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4a98: 0x7f9493e88cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 139 (Thread 0x7f945ec8d640 (LWP 2739744) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e887c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e887c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4a68: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4a78: 0x7f9493e887c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4a60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 138 (Thread 0x7f945f68e640 (LWP 2739743) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4a48: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4a58: 0x7f9493e88740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4a40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 137 (Thread 0x7f946008f640 (LWP 2739742) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4a28: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4a38: 0x7f9493e88600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4a20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 136 (Thread 0x7f9460a90640 (LWP 2739741) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e881c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e881c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4a08: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4a18: 0x7f9493e881c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 135 (Thread 0x7f9461491640 (LWP 2739740) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb49e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb49f8: 0x7f9493e88140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb49e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 134 (Thread 0x7f9461e92640 (LWP 2739739) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88000) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88000) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb49c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb49d8: 0x7f9493e88000) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb49c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 133 (Thread 0x7f9462893640 (LWP 2739738) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6cec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6cec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb49a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb49b8: 0x7f9493e6cec0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb49a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 132 (Thread 0x7f9463294640 (LWP 2739737) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c9c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c9c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4988: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4998: 0x7f9493e6c9c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4980) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 131 (Thread 0x7f9463c95640 (LWP 2739736) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c940) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c940) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4968: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4978: 0x7f9493e6c940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4960) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 130 (Thread 0x7f9464696640 (LWP 2739735) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c8c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c8c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4948: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4958: 0x7f9493e6c8c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 129 (Thread 0x7f9465097640 (LWP 2739734) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c6c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c6c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4928: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4938: 0x7f9493e6c6c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4920) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 128 (Thread 0x7f9465a98640 (LWP 2739733) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4908: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4918: 0x7f9493e6c640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 127 (Thread 0x7f9466499640 (LWP 2739732) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb48e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb48f8: 0x7f9493e6c440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb48e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 126 (Thread 0x7f9466e9a640 (LWP 2739731) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c300) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c300) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb48c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb48d8: 0x7f9493e6c300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb48c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 125 (Thread 0x7f946789b640 (LWP 2739730) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c280) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c280) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb48a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb48b8: 0x7f9493e6c280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb48a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 124 (Thread 0x7f946829c640 (LWP 2739729) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c080) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c080) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4888: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4898: 0x7f9493e6c080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4880) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 123 (Thread 0x7f9468c9d640 (LWP 2739728) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4868: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4878: 0x7f9493e14fc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4860) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 122 (Thread 0x7f946969e640 (LWP 2739727) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14d00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14d00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4848: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4858: 0x7f9493e14d00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 121 (Thread 0x7f946a09f640 (LWP 2739726) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14bc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14bc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4828: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4838: 0x7f9493e14bc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4820) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 120 (Thread 0x7f946aaa0640 (LWP 2739725) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14b40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14b40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4808: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4818: 0x7f9493e14b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4800) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 119 (Thread 0x7f946b4a1640 (LWP 2739724) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14a00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14a00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb47e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb47f8: 0x7f9493e14a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb47e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 118 (Thread 0x7f946bea2640 (LWP 2739723) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e148c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e148c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb47c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb47d8: 0x7f9493e148c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb47c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 117 (Thread 0x7f946c8a3640 (LWP 2739722) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14840) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14840) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb47a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb47b8: 0x7f9493e14840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb47a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 116 (Thread 0x7f946d2a4640 (LWP 2739721) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4788: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4798: 0x7f9493e14340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 115 (Thread 0x7f946dca5640 (LWP 2739720) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14200) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14200) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4768: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4778: 0x7f9493e14200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4760) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 114 (Thread 0x7f946e6a6640 (LWP 2739719) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14180) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14180) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4748: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4758: 0x7f9493e14180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 113 (Thread 0x7f946f0a7640 (LWP 2739718) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df8540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df8540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4728: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4738: 0x7f9493df8540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 112 (Thread 0x7f946faa8640 (LWP 2739717) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4708: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4718: 0x7f9493df3fc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4700) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 111 (Thread 0x7f94704a9640 (LWP 2739716) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3dc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3dc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb46e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb46f8: 0x7f9493df3dc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb46e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 110 (Thread 0x7f9470eaa640 (LWP 2739715) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3bc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3bc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb46c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb46d8: 0x7f9493df3bc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb46c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 109 (Thread 0x7f94718ab640 (LWP 2739714) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3b40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3b40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb46a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb46b8: 0x7f9493df3b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb46a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 108 (Thread 0x7f94722ac640 (LWP 2739713) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df37c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df37c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4688: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4698: 0x7f9493df37c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 107 (Thread 0x7f9472cad640 (LWP 2739712) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3680) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3680) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4668: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4678: 0x7f9493df3680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4660) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 106 (Thread 0x7f94736ae640 (LWP 2739711) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4648: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4658: 0x7f9493df3600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 105 (Thread 0x7f94740af640 (LWP 2739710) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df34c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df34c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4628: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4638: 0x7f9493df34c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4620) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 104 (Thread 0x7f9474ab0640 (LWP 2739709) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4608: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4618: 0x7f9493df3440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 103 (Thread 0x7f94754b1640 (LWP 2739708) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df33c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df33c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb45e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb45f8: 0x7f9493df33c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb45e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 102 (Thread 0x7f9475eb2640 (LWP 2739707) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb45c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb45d8: 0x7f9493df3340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb45c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 101 (Thread 0x7f94768b3640 (LWP 2739706) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb45a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb45b8: 0x7f9493df3040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb45a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 100 (Thread 0x7f94772b4640 (LWP 2739705) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493deeec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493deeec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4588: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4598: 0x7f9493deeec0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 99 (Thread 0x7f9477cb5640 (LWP 2739704) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493deed00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493deed00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4568: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4578: 0x7f9493deed00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4560) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 98 (Thread 0x7f94786b6640 (LWP 2739703) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dee880) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dee880) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4548: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4558: 0x7f9493dee880) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 97 (Thread 0x7f94790b7640 (LWP 2739702) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dee6c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dee6c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4528: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4538: 0x7f9493dee6c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4520) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 96 (Thread 0x7f9479ab8640 (LWP 2739701) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dee300) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dee300) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4508: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4518: 0x7f9493dee300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 95 (Thread 0x7f947a4b9640 (LWP 2739700) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dead80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dead80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb44e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb44f8: 0x7f9493dead80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb44e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 94 (Thread 0x7f947aeba640 (LWP 2739699) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493deabc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493deabc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb44c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb44d8: 0x7f9493deabc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb44c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 93 (Thread 0x7f947b8bb640 (LWP 2739698) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dea5c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dea5c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb44a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb44b8: 0x7f9493dea5c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb44a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 92 (Thread 0x7f947c2bc640 (LWP 2739697) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dea2c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dea2c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4488: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4498: 0x7f9493dea2c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 91 (Thread 0x7f947ccbd640 (LWP 2739696) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dea140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dea140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4468: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4478: 0x7f9493dea140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4460) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 90 (Thread 0x7f947d6be640 (LWP 2739695) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493de2b00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493de2b00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4448: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4458: 0x7f9493de2b00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 89 (Thread 0x7f947e0bf640 (LWP 2739694) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493de2540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493de2540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4428: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4438: 0x7f9493de2540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4420) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 88 (Thread 0x7f947eac0640 (LWP 2739693) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493de2480) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493de2480) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4408: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4418: 0x7f9493de2480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 87 (Thread 0x7f947f4c1640 (LWP 2739692) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938edb80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938edb80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb43e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb43f8: 0x7f94938edb80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb43e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 86 (Thread 0x7f947fec2640 (LWP 2739691) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938edb00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938edb00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb43c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb43d8: 0x7f94938edb00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb43c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 85 (Thread 0x7f94808c3640 (LWP 2739690) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb43a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb43b8: 0x7f94938ed900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb43a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 84 (Thread 0x7f94812c4640 (LWP 2739689) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed7c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed7c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4388: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4398: 0x7f94938ed7c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 83 (Thread 0x7f9481cc5640 (LWP 2739688) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4368: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4378: 0x7f94938ed740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 82 (Thread 0x7f94826c6640 (LWP 2739687) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed3c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed3c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4348: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4358: 0x7f94938ed3c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 81 (Thread 0x7f94830c7640 (LWP 2739686) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed1c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed1c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4328: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4338: 0x7f94938ed1c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 80 (Thread 0x7f9483ac8640 (LWP 2739685) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebff600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebff600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4308: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4318: 0x7f956ebff600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 79 (Thread 0x7f94844c9640 (LWP 2739684) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb42e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb42f8: 0x7f94938ed040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb42e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 78 (Thread 0x7f9484eca640 (LWP 2739683) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebfff40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebfff40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb42c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb42d8: 0x7f956ebfff40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb42c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 77 (Thread 0x7f94858cb640 (LWP 2739682) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebffbc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebffbc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb42a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb42b8: 0x7f956ebffbc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb42a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 76 (Thread 0x7f94862cc640 (LWP 2739681) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebffa40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebffa40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94bd819068: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94bd819078: 0x7f956ebffa40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94bd819060) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 75 (Thread 0x7f9486ccd640 (LWP 2739680) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebff580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebff580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f956eb788e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f956eb788f8: 0x7f956ebff580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f956eb788e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 74 (Thread 0x7f94876ce640 (LWP 2739679) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebfed80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebfed80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f956ebeac48: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f956ebeac58: 0x7f956ebfed80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f956ebeac40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 73 (Thread 0x7f94880cf640 (LWP 2739678) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebfea80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebfea80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94bd819088: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94bd819098: 0x7f956ebfea80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94bd819080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 72 (Thread 0x7f9488ad0640 (LWP 2739677) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebfe900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebfe900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4228: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4238: 0x7f956ebfe900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 71 (Thread 0x7f94894d1640 (LWP 2739676) "SegmentReader"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 +#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebfe5c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 +#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebfmpptaske5c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 +#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94bd819008: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94bd819018: 0x7f956ebfe5c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94bd819000) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 70 (Thread 0x7f9489ed2640 (LWP 2739675) "MPPTask-Moniter"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f9493f9c900, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abcc52f in std::__1::condition_variable::wait_for > (this=0x7f9493f9c900, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c26907e86 in DB::(anonymous namespace)::monitorMPPTasks (monitor=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/TMTContext.cpp:103 +#5 0x0000558c26907dd6 in DB::(anonymous namespace)::startMonitorMPPTaskThread(std::__1::shared_ptr const&)::$_0::operator()() const (this=0x7f956f9af1a8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/TMTContext.cpp:123 +#6 0x0000558c26907da5 in std::__1::__invoke[abi:ue170006] const&)::$_0&>(DB::(anonymous namespace)::startMonitorMPPTaskThread(std::__1::shared_ptr const&)::$_0&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c26907d65 in std::__1::__invoke_void_return_wrapper::__call[abi:ue170006] const&)::$_0&>(DB::(anonymous namespace)::startMonitorMPPTaskThread(std::__1::shared_ptr const&)::$_0&) (__args=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:415 +#8 0x0000558c26907d3d in std::__1::__function::__alloc_func const&)::$_0, std::__1::allocator const&)::$_0>, void ()>::operator()[abi:ue170006]() (this=0x7f956f9af1a8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:192 +#9 0x0000558c26907039 in std::__1::__function::__func const&)::$_0, std::__1::allocator const&)::$_0>, void ()>::operator()() (this=0x7f956f9af1a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:363 +#10 0x0000558c1abb48d2 in std::__1::__function::__value_func::operator()[abi:ue170006]() const (this=0x7f956f9af1a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:517 +#11 0x0000558c1aba7925 in std::__1::function::operator()() const (this=0x7f956f9af1a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:1168 +#12 0x0000558c238349d5 in std::__1::__invoke[abi:ue170006] const&>(std::__1::function const&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#13 0x0000558c23834975 in std::__1::invoke[abi:ue170006] const&>(std::__1::function const&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 +#14 0x0000558c238348f3 in DB::ThreadFactory::newThread>(bool, std::__1::basic_string, std::__1::allocator >, std::__1::function&&)::{lambda((auto:1&&)...)#1}::operator()<>() const (this=0x7f956f9af170) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 +#15 0x0000558c23834775 in std::__1::__invoke[abi:ue170006]>(bool, std::__1::basic_string, std::__1::allocator >, std::__1::function&&)::{lambda((auto:1&&)...)#1}>(std::__1::function&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#16 0x0000558c2383474d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread>(bool, std::__1::basic_string, std::__1::allocator >, std::__1::function&&)::{lambda((auto:1&&)...)#1}>(std::__1::tuple >, DB::ThreadFactory::newThread>(bool, std::__1::basic_string, std::__1::allocator >, std::__1::function&&)::{lambda((auto:1&&)...)#1}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#17 0x0000558c238344d2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread>(bool, std::__1::basic_string, std::__1::allocator >, std::__1::function&&)::{lambda((auto:1&&)...)#1}> >(void*) (__vp=0x7f956f9af160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 69 (Thread 0x7f948a8d3640 (LWP 2739674) "TiFlashMain"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f872688, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f956f872688, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c27b5afe9 in std::__1::condition_variable::wait_until >, pingcap::common::MPPProber::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, pingcap::common::MPPProber::run()::$_0) (this=0x7f956f872688, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c27b5a59f in std::__1::condition_variable::wait_for[abi:ue170006], pingcap::common::MPPProber::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, pingcap::common::MPPProber::run()::$_0) (this=0x7f956f872688, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 +#6 0x0000558c27b5a4f5 in pingcap::common::MPPProber::run (this=0x7f956f8725e0) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/common/MPPProber.cc:53 +#7 0x0000558c27ae93f4 in pingcap::kv::Cluster::startBackgroundTasks()::$_0::operator()() const (this=0x7f948a89fba8) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/kv/Cluster.cc:34 +#8 0x0000558c27ae93c5 in std::__1::__invoke[abi:ue170006](pingcap::kv::Cluster::startBackgroundTasks()::$_0&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#9 0x0000558c27ae9385 in std::__1::__invoke_void_return_wrapper::__call[abi:ue170006](pingcap::kv::Cluster::startBackgroundTasks()::$_0&) (__args=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:415 +#10 0x0000558c27ae935d in std::__1::__function::__alloc_func, void ()>::operator()[abi:ue170006]() (this=0x7f948a89fba8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:192 +#11 0x0000558c27ae8769 in std::__1::__function::__func, void ()>::operator()() (this=0x7f948a89fba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:363 +#12 0x0000558c1abb48d2 in std::__1::__function::__value_func::operator()[abi:ue170006]() const (this=0x7f948a89fba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:517 +#13 0x0000558c1aba7925 in std::__1::function::operator()() const (this=0x7f948a89fba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:1168 +#14 0x0000558c27b593af in pingcap::common::FixedThreadPool::loop (this=0x7f9493f9ddc0) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/common/FixedThreadPool.cc:33 +#15 0x0000558c27b59ae9 in std::__1::__invoke[abi:ue170006](void (pingcap::common::FixedThreadPool::*&&)(), pingcap::common::FixedThreadPool*&&) (__f=@0x7f956ebf3008: (void (pingcap::common::FixedThreadPool::*)(class pingcap::common::FixedThreadPool * const)) 0x558c27b592b0 , __a0=@0x7f956ebf3018: 0x7f9493f9ddc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#16 0x0000558c27b59a6e in std::__1::__thread_execute[abi:ue170006] >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*, 2ul>(std::__1::tuple >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#17 0x0000558c27b59842 in std::__1::__thread_proxy[abi:ue170006] >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*> >(void*) (__vp=0x7f956ebf3000) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 68 (Thread 0x7f948b2d4640 (LWP 2739673) "TiFlashMain"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f8c33b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f956f8c33b0, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c27aec349 in std::__1::condition_variable::wait_until >, pingcap::kv::RegionCache::updateCachePeriodically()::{lambda()#1}>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, pingcap::kv::RegionCache::updateCachePeriodically()::{lambda()#1}) (this=0x7f956f8c33b0, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c27aec2ef in std::__1::condition_variable::wait_for[abi:ue170006], pingcap::kv::RegionCache::updateCachePeriodically()::{lambda()#1}>(std::__1::unique_lock&, std::__1::chrono::duration > const&, pingcap::kv::RegionCache::updateCachePeriodically()::{lambda()#1}) (this=0x7f956f8c33b0, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 +#6 0x0000558c27aec24d in pingcap::kv::RegionCache::updateCachePeriodically (this=0x7f956f8c3200) at /DATA/disk3/xzx/tiflash/contrib/client-c/include/pingcap/kv/RegionCache.h:244 +#7 0x0000558c27aea4f4 in pingcap::kv::Cluster::startBackgroundTasks()::$_1::operator()() const (this=0x7f948b2a0ba8) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/kv/Cluster.cc:41 +#8 0x0000558c27aea4c5 in std::__1::__invoke[abi:ue170006](pingcap::kv::Cluster::startBackgroundTasks()::$_1&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#9 0x0000558c27aea485 in std::__1::__invoke_void_return_wrapper::__call[abi:ue170006](pingcap::kv::Cluster::startBackgroundTasks()::$_1&) (__args=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:415 +#10 0x0000558c27aea45d in std::__1::__function::__alloc_func, void ()>::operator()[abi:ue170006]() (this=0x7f948b2a0ba8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:192 +#11 0x0000558c27ae9869 in std::__1::__function::__func, void ()>::operator()() (this=0x7f948b2a0ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:363 +#12 0x0000558c1abb48d2 in std::__1::__function::__value_func::operator()[abi:ue170006]() const (this=0x7f948b2a0ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:517 +#13 0x0000558c1aba7925 in std::__1::function::operator()() const (this=0x7f948b2a0ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:1168 +#14 0x0000558c27b593af in pingcap::common::FixedThreadPool::loop (this=0x7f9493f9ddc0) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/common/FixedThreadPool.cc:33 +#15 0x0000558c27b59ae9 in std::__1::__invoke[abi:ue170006](void (pingcap::common::FixedThreadPool::*&&)(), pingcap::common::FixedThreadPool*&&) (__f=@0x7f956ebeacc8: (void (pingcap::common::FixedThreadPool::*)(class pingcap::common::FixedThreadPool * const)) 0x558c27b592b0 , __a0=@0x7f956ebeacd8: 0x7f9493f9ddc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 +#16 0x0000558c27b59a6e in std::__1::__thread_execute[abi:ue170006] >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*, 2ul>(std::__1::tuple >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#17 0x0000558c27b59842 in std::__1::__thread_proxy[abi:ue170006] >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*> >(void*) (__vp=0x7f956ebeacc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 67 (Thread 0x7f948bcd5640 (LWP 2739672) "PDUpdateTS"): +#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 +#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 +#2 0x00007f95847adc8b in std::__1::this_thread::sleep_for(std::__1::chrono::duration > const&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ab783b2 in std::__1::this_thread::sleep_for[abi:ue170006] >(std::__1::chrono::duration > const&) (__d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/this_thread.h:55 +#4 0x0000558c2691b43d in pingcap::pd::Oracle::updateTS (this=0x7f956ebf79c0, update_interval=...) at /DATA/disk3/xzx/tiflash/contrib/client-c/include/pingcap/pd/Oracle.h:79 +#5 0x0000558c2691b324 in pingcap::pd::Oracle::Oracle(std::__1::shared_ptr, std::__1::chrono::duration >)::{lambda()#1}::operator()() const (this=0x7f9493def088) at /DATA/disk3/xzx/tiflash/contrib/client-c/include/pingcap/pd/Oracle.h:42 +#6 0x0000558c2691b2d5 in std::__1::__invoke[abi:ue170006], std::__1::chrono::duration >)::{lambda()#1}>(pingcap::pd::Oracle::Oracle(std::__1::shared_ptr, std::__1::chrono::duration >)::{lambda()#1}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c2691b2ad in std::__1::__thread_execute[abi:ue170006] >, pingcap::pd::Oracle::Oracle(std::__1::shared_ptr, std::__1::chrono::duration >)::{lambda()#1}>(std::__1::tuple >, pingcap::pd::Oracle::Oracle(std::__1::shared_ptr, std::__1::chrono::duration >)::{lambda()#1}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c2691b0e2 in std::__1::__thread_proxy[abi:ue170006] >, pingcap::pd::Oracle::Oracle(std::__1::shared_ptr, std::__1::chrono::duration >)::{lambda()#1}> >(void*) (__vp=0x7f9493def080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 66 (Thread 0x7f948c6d6640 (LWP 2739671) "PDLeaderLoop"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f957068996f in pthread_cond_timedwait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799cda in std::__1::condition_variable::__do_timed_wait(std::__1::unique_lock&, std::__1::chrono::time_point > >) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c27b346bb in std::__1::condition_variable::wait_until > > (this=0x7f95702af738, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c27b26ea9 in std::__1::condition_variable::wait_until >, pingcap::pd::Client::leaderLoop()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, pingcap::pd::Client::leaderLoop()::$_0) (this=0x7f95702af738, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c27b26c31 in pingcap::pd::Client::leaderLoop (this=0x7f95702af598) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/pd/Client.cc:254 +#6 0x0000558c27b2bce8 in pingcap::pd::Client::Client(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0::operator()() const (this=0x7f94bd815018) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/pd/Client.cc:56 +#7 0x0000558c27b2bca5 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0>(pingcap::pd::Client::Client(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c27b2bc7d in std::__1::__thread_execute[abi:ue170006] >, pingcap::pd::Client::Client(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0>(std::__1::tuple >, pingcap::pd::Client::Client(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c27b2bab2 in std::__1::__thread_proxy[abi:ue170006] >, pingcap::pd::Client::Client(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0> >(void*) (__vp=0x7f94bd815010) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 65 (Thread 0x7f948d0d7640 (LWP 2739670) "grpc_global_tim"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x0000558c288f9efe in absl::lts_20211102::synchronization_internal::FutexImpl::WaitUntil (v=0x7f958456cf40, val=0, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/futex.h:104 +#2 0x0000558c288f9bfb in absl::lts_20211102::synchronization_internal::Waiter::Wait (this=0x7f958456cf40, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/waiter.cc:95 +#3 0x0000558c288f9816 in AbslInternalPerThreadSemWait_lts_20211102 (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.cc:93 +#4 0x0000558c288f6bbd in absl::lts_20211102::synchronization_internal::PerThreadSem::Wait (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.h:112 +#5 0x0000558c288efd56 in absl::lts_20211102::Mutex::DecrementSynchSem (mu=0x558c29567628 , w=0x7f958456cf00, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:579 +#6 0x0000558c288f4e7a in absl::lts_20211102::CondVar::WaitCommon (this=0x558c29567630 , mutex=0x558c29567628 , t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2559 +#7 0x0000558c288f50aa in absl::lts_20211102::CondVar::Wait (this=0x558c29567630 , mu=0x558c29567628 ) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2594 +#8 0x0000558c2832ff8b in gpr_cv_wait (cv=0x558c29567630 , mu=0x558c29567628 , abs_deadline=...) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gpr/sync_abseil.cc:85 +#9 0x0000558c27e19621 in wait_until (next=9223372036854775807) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:200 +#10 0x0000558c27e191e9 in timer_main_loop () at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:255 +#11 0x0000558c27e190ef in timer_thread (completed_thread_ptr=0x7f9493e250e0) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:284 +#12 0x0000558c28339f22 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::operator()(void*) const (this=0x7f948d0a3c87, v=0x7f9493e25140) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:140 +#13 0x0000558c28339e09 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) (v=0x7f9493e25140) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:110 +#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 64 (Thread 0x7f948dad8640 (LWP 2739669) "resolver-execut"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x0000558c288f9efe in absl::lts_20211102::synchronization_internal::FutexImpl::WaitUntil (v=0x7f958456c940, val=0, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/futex.h:104 +#2 0x0000558c288f9bfb in absl::lts_20211102::synchronization_internal::Waiter::Wait (this=0x7f958456c940, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/waiter.cc:95 +#3 0x0000558c288f9816 in AbslInternalPerThreadSemWait_lts_20211102 (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.cc:93 +#4 0x0000558c288f6bbd in absl::lts_20211102::synchronization_internal::PerThreadSem::Wait (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.h:112 +#5 0x0000558c288efd56 in absl::lts_20211102::Mutex::DecrementSynchSem (mu=0x7f9493fa6800, w=0x7f958456c900, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:579 +#6 0x0000558c288f4e7a in absl::lts_20211102::CondVar::WaitCommon (this=0x7f9493fa6818, mutex=0x7f9493fa6800, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2559 +#7 0x0000558c288f50aa in absl::lts_20211102::CondVar::Wait (this=0x7f9493fa6818, mu=0x7f9493fa6800) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2594 +#8 0x0000558c2832ff8b in gpr_cv_wait (cv=0x7f9493fa6818, mu=0x7f9493fa6800, abs_deadline=...) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gpr/sync_abseil.cc:85 +#9 0x0000558c27df194f in grpc_core::Executor::ThreadMain (arg=0x7f9493fa6800) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/executor.cc:238 +#10 0x0000558c28339f22 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::operator()(void*) const (this=0x7f948daa4c87, v=0x7f9493e25080) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:140 +#11 0x0000558c28339e09 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) (v=0x7f9493e25080) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:110 +#12 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#13 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 63 (Thread 0x7f948e4d9640 (LWP 2739668) "default-executo"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x0000558c288f9efe in absl::lts_20211102::synchronization_internal::FutexImpl::WaitUntil (v=0x7f958456c340, val=0, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/futex.h:104 +#2 0x0000558c288f9bfb in absl::lts_20211102::synchronization_internal::Waiter::Wait (this=0x7f958456c340, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/waiter.cc:95 +#3 0x0000558c288f9816 in AbslInternalPerThreadSemWait_lts_20211102 (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.cc:93 +#4 0x0000558c288f6bbd in absl::lts_20211102::synchronization_internal::PerThreadSem::Wait (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.h:112 +#5 0x0000558c288efd56 in absl::lts_20211102::Mutex::DecrementSynchSem (mu=0x7f9493fa3000, w=0x7f958456c300, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:579 +#6 0x0000558c288f4e7a in absl::lts_20211102::CondVar::WaitCommon (this=0x7f9493fa3018, mutex=0x7f9493fa3000, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2559 +#7 0x0000558c288f50aa in absl::lts_20211102::CondVar::Wait (this=0x7f9493fa3018, mu=0x7f9493fa3000) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2594 +#8 0x0000558c2832ff8b in gpr_cv_wait (cv=0x7f9493fa3018, mu=0x7f9493fa3000, abs_deadline=...) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gpr/sync_abseil.cc:85 +#9 0x0000558c27df194f in grpc_core::Executor::ThreadMain (arg=0x7f9493fa3000) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/executor.cc:238 +#10 0x0000558c28339f22 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::operator()(void*) const (this=0x7f948e4a5c87, v=0x7f9493e25020) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:140 +#11 0x0000558c28339e09 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) (v=0x7f9493e25020) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:110 +#12 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#13 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 62 (Thread 0x7f948eeda640 (LWP 2739667) "TiFlashMain"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f95702af4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f95702af4b0, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c238923c9 in std::__1::condition_variable::wait_until >, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0) (this=0x7f95702af4b0, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c23891fff in std::__1::condition_variable::wait_for[abi:ue170006], DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0) (this=0x7f95702af4b0, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 +#6 0x0000558c23891ea0 in DB::DM::LocalIndexCache::cleanOutdatedLoop (this=0x7f95702af318) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/Index/LocalIndexCache.cpp:69 +#7 0x0000558c23892738 in DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0::operator()() const (this=0x7f9493def038) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/Index/LocalIndexCache.cpp:90 +#8 0x0000558c238926f5 in std::__1::__invoke[abi:ue170006](DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#9 0x0000558c238926cd in std::__1::__thread_execute[abi:ue170006] >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0>(std::__1::tuple >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#10 0x0000558c23892502 in std::__1::__thread_proxy[abi:ue170006] >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0> >(void*) (__vp=0x7f9493def030) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#11 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#12 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 61 (Thread 0x7f948f8db640 (LWP 2739666) "TiFlashMain"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f95702af230, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f95702af230, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c238923c9 in std::__1::condition_variable::wait_until >, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0) (this=0x7f95702af230, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c23891fff in std::__1::condition_variable::wait_for[abi:ue170006], DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0) (this=0x7f95702af230, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 +#6 0x0000558c23891ea0 in DB::DM::LocalIndexCache::cleanOutdatedLoop (this=0x7f95702af098) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/Index/LocalIndexCache.cpp:69 +#7 0x0000558c23892738 in DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0::operator()() const (this=0x7f9493def028) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/Index/LocalIndexCache.cpp:90 +#8 0x0000558c238926f5 in std::__1::__invoke[abi:ue170006](DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#9 0x0000558c238926cd in std::__1::__thread_execute[abi:ue170006] >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0>(std::__1::tuple >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#10 0x0000558c23892502 in std::__1::__thread_proxy[abi:ue170006] >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0> >(void*) (__vp=0x7f9493def020) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#11 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#12 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 60 (Thread 0x7f94902dc640 (LWP 2739665) "TiFlashMain"): +#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 +#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 +#2 0x00007f95847adc8b in std::__1::this_thread::sleep_for(std::__1::chrono::duration > const&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c1ab783b2 in std::__1::this_thread::sleep_for[abi:ue170006] >(std::__1::chrono::duration > const&) (__d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/this_thread.h:55 +#4 0x0000558c1ac1aafa in DB::IORateLimiter::runAutoTune()::$_0::operator()() const (this=0x7f9493def018) at /DATA/disk3/xzx/tiflash/dbms/src/IO/BaseFile/RateLimiter.cpp:693 +#5 0x0000558c1ac1aa65 in std::__1::__invoke[abi:ue170006](DB::IORateLimiter::runAutoTune()::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#6 0x0000558c1ac1aa3d in std::__1::__thread_execute[abi:ue170006] >, DB::IORateLimiter::runAutoTune()::$_0>(std::__1::tuple >, DB::IORateLimiter::runAutoTune()::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#7 0x0000558c1ac1a872 in std::__1::__thread_proxy[abi:ue170006] >, DB::IORateLimiter::runAutoTune()::$_0> >(void*) (__vp=0x7f9493def010) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#8 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#9 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 59 (Thread 0x7f9492cdd640 (LWP 2739664) "LocalIndexSched"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c22d9cd31 in std::__1::condition_variable::wait(std::__1::unique_lock&, DB::DM::LocalIndexerScheduler::schedulerLoop()::$_0) (this=0x7f956f065c58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 +#4 0x0000558c22d9cb46 in DB::DM::LocalIndexerScheduler::schedulerLoop (this=0x7f956f065b98) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.cpp:412 +#5 0x0000558c22d9d048 in DB::DM::LocalIndexerScheduler::start()::$_0::operator()() const (this=0x7f9570296fb8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.cpp:102 +#6 0x0000558c22d9d005 in std::__1::__invoke[abi:ue170006](DB::DM::LocalIndexerScheduler::start()::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#7 0x0000558c22d9cfdd in std::__1::__thread_execute[abi:ue170006] >, DB::DM::LocalIndexerScheduler::start()::$_0>(std::__1::tuple >, DB::DM::LocalIndexerScheduler::start()::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#8 0x0000558c22d9ce12 in std::__1::__thread_proxy[abi:ue170006] >, DB::DM::LocalIndexerScheduler::start()::$_0> >(void*) (__vp=0x7f9570296fb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 58 (Thread 0x7f94949de640 (LWP 2739663) "bg-block-15"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=15) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bfc8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bfc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 57 (Thread 0x7f94953df640 (LWP 2739662) "bg-block-14"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=14) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bfa8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bfa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 56 (Thread 0x7f9495de0640 (LWP 2739661) "bg-block-13"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=13) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bf88) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bf80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 55 (Thread 0x7f94967e1640 (LWP 2739660) "bg-block-12"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=12) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bf68) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bf60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 54 (Thread 0x7f94971e2640 (LWP 2739659) "bg-block-11"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=11) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bf48) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bf40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 53 (Thread 0x7f9497be3640 (LWP 2739658) "bg-block-10"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=10) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bf28) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bf20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 52 (Thread 0x7f94985e4640 (LWP 2739657) "bg-block-9"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=9) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bf08) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bf00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 51 (Thread 0x7f9498fe5640 (LWP 2739656) "bg-block-8"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bee8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bee0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 50 (Thread 0x7f94999e6640 (LWP 2739655) "bg-block-7"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=7) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bec8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bec0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 49 (Thread 0x7f949a3e7640 (LWP 2739654) "bg-block-6"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=6) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bea8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 48 (Thread 0x7f949ade8640 (LWP 2739653) "bg-block-5"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=5) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6be88) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6be80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 47 (Thread 0x7f949b7e9640 (LWP 2739652) "bg-block-4"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=4) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6be68) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6be60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 46 (Thread 0x7f949c1ea640 (LWP 2739651) "bg-block-3"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=3) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6be48) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6be40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 45 (Thread 0x7f949cbeb640 (LWP 2739650) "bg-block-2"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=2) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6be28) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6be20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 44 (Thread 0x7f949d5ec640 (LWP 2739649) "bg-block-1"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=1) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6be08) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6be00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 43 (Thread 0x7f949dfed640 (LWP 2739648) "bg-block-0"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bde8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bde0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 42 (Thread 0x7f949e9ee640 (LWP 2739647) "bg-15"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=15) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bdc8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bdc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 41 (Thread 0x7f949f3ef640 (LWP 2739646) "bg-14"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=14) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bda8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bda0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 40 (Thread 0x7f949fdf0640 (LWP 2739645) "bg-13"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=13) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bd88) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bd80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 39 (Thread 0x7f94a07f1640 (LWP 2739644) "bg-12"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=12) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bd68) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bd60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 38 (Thread 0x7f94a11f2640 (LWP 2739643) "bg-11"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=11) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bd48) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bd40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 37 (Thread 0x7f94a1bf3640 (LWP 2739642) "bg-10"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=10) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bd28) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bd20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 36 (Thread 0x7f94a25f4640 (LWP 2739641) "bg-9"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=9) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bd08) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bd00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 35 (Thread 0x7f94a2ff5640 (LWP 2739640) "bg-8"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bce8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bce0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 34 (Thread 0x7f94a39f6640 (LWP 2739639) "bg-7"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=7) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bcc8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bcc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 33 (Thread 0x7f94a43f7640 (LWP 2739638) "bg-6"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=6) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bca8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bca0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 32 (Thread 0x7f94a4df8640 (LWP 2739637) "bg-5"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=5) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bc88) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bc80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 31 (Thread 0x7f94a57f9640 (LWP 2739636) "bg-4"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=4) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bc68) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bc60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 30 (Thread 0x7f94a61fa640 (LWP 2739635) "bg-3"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=3) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bc48) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bc40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 29 (Thread 0x7f94a6bfb640 (LWP 2739634) "bg-2"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=2) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bc28) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bc20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 28 (Thread 0x7f94a75fc640 (LWP 2739633) "bg-1"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=1) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bc08) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bc00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 27 (Thread 0x7f94a7ffd640 (LWP 2739632) "bg-0"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 +#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 +#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 +#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6b9a8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 +#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6b9a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 26 (Thread 0x7f94a89fe640 (LWP 2739631) "ThdMemTrace"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 +#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f957025c628, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 +#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f957025c628, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 +#4 0x0000558c2696ead9 in std::__1::condition_variable::wait_until >, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0::operator()() const::{lambda()#1}>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0::operator()() const::{lambda()#1}) (this=0x7f957025c628, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 +#5 0x0000558c2696ea7f in std::__1::condition_variable::wait_for[abi:ue170006], DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0::operator()() const::{lambda()#1}>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0::operator()() const::{lambda()#1}) (this=0x7f957025c628, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 +#6 0x0000558c2696e99d in DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0::operator()() const (this=0x7f957020efd8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/FFI/JointThreadAllocInfo.cpp:39 +#7 0x0000558c2696e8f5 in std::__1::__invoke[abi:ue170006](DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c2696e8cd in std::__1::__thread_execute[abi:ue170006] >, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0>(std::__1::tuple >, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c2696e702 in std::__1::__thread_proxy[abi:ue170006] >, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0> >(void*) (__vp=0x7f957020efd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 25 (Thread 0x7f94ab1ff640 (LWP 2739630) "grpc_global_tim"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f958386d39d in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f95838674f0 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f9583917710 in timer_thread(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 24 (Thread 0x7f94d05ff640 (LWP 2739555) "background-3"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94d05fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94d05fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94d05fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689431152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94d05cac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94d05cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94d05cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 23 (Thread 0x7f94d0dff640 (LWP 2739554) "background-2"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94d0dfb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94d0dfb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94d0dfb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689431152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94d0dcac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94d0dcab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94d0dcb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 22 (Thread 0x7f94d13fe640 (LWP 2739553) "background-1"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94d13fa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94d13fa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94d13fa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689431152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94d13c9c70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94d13c9b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94d13ca170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 21 (Thread 0x7f94d15ff640 (LWP 2739552) "background-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94d15fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94d15fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94d15fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689431152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94d15cac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94d15cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94d15cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 20 (Thread 0x7f95625ff640 (LWP 2739550) "timer"): +#0 0x00007f9580aaadda in futures_channel::mpsc::UnboundedSenderInner>::inc_num_messages> (self=0x7f95625caff0) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-channel-0.3.31/src/mpsc/mod.rs:448 +#1 0x00007f9580aa9f11 in futures_channel::mpsc::UnboundedSender>::do_send_nb> (self=0x7f95625caff0, msg=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-channel-0.3.31/src/mpsc/mod.rs:793 +#2 0x00007f9580aaa0a7 in futures_channel::mpsc::UnboundedSender>::unbounded_send> (self=0x7f95625caff0, msg=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-channel-0.3.31/src/mpsc/mod.rs:816 +#3 0x00007f9580ae9b5a in tikv_util::worker::pool::{impl#12}::delay_notify::{closure#0} () at components/tikv_util/src/worker/pool.rs:462 +#4 0x00007f9580ade44f in futures_util::fns::{impl#0}::call_once, core::result::Result<(), tokio_timer::error::Error>, ()> (self=..., arg=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-util-0.3.31/src/fns.rs:15 +#5 0x00007f9580ad76a3 in futures_util::future::future::map::{impl#2}::poll, tikv_util::worker::pool::{impl#12}::delay_notify::{closure_env#0}, ()> (self=..., cx=0x7f95625cb260) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-util-0.3.31/src/future/future/map.rs:57 +#6 0x00007f9580ae0b75 in futures_util::future::future::{impl#15}::poll, tikv_util::worker::pool::{impl#12}::delay_notify::{closure_env#0}> (self=..., cx=0x7f95625cb260) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-util-0.3.31/src/lib.rs:86 +#7 0x00007f9582810696 in tikv_util::future::PollAtWake::poll (arc_self=0x7f95625cb3e8) at components/tikv_util/src/future.rs:156 +#8 0x00007f958281091b in tikv_util::future::{impl#3}::wake_by_ref (arc_self=0x7f95625cb3e8) at components/tikv_util/src/future.rs:200 +#9 0x00007f95828246cc in futures_task::waker::wake_by_ref_arc_raw (data=0x7f94bf062320) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-task-0.3.31/src/waker.rs:55 +#10 0x00007f9583825482 in core::task::wake::Waker::wake_by_ref (self=0x7f94bf0195e0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/task/wake.rs:300 +#11 0x00007f9583825ef3 in futures_util::compat::compat01as03::{impl#12}::notify (self=0x7f94bf0195e0) at src/compat/compat01as03.rs:339 +#12 0x00007f958385973b in futures::task_impl::NotifyHandle::notify (self=0x7f95625cb560, id=0) at src/task_impl/mod.rs:657 +#13 0x00007f9583856c98 in futures::task_impl::core::TaskUnpark::notify (self=0x7f95625cb558) at src/task_impl/core.rs:67 +#14 0x00007f958385387e in futures::task_impl::std::TaskUnpark::notify (self=0x7f95625cb558) at src/task_impl/std/mod.rs:204 +#15 0x00007f9583859711 in futures::task_impl::Task::notify (self=0x7f95625cb530) at src/task_impl/mod.rs:147 +#16 0x00007f9583859388 in futures::task_impl::atomic_task::AtomicTask::notify (self=0x7f94bf11e520) at src/task_impl/atomic_task.rs:250 +#17 0x00007f9582a00d83 in tokio_timer::timer::entry::Entry::fire (self=0x7f94bf11e480, when=57319452) at src/timer/entry.rs:251 +#18 0x00007f95828a5574 in tokio_timer::timer::Timer::process (self=0x7f95625cb7d8) at /home/xzx/.cargo/git/checkouts/tokio-8e927faba632ed16/4394380/tokio-timer/src/timer/mod.rs:276 +#19 0x00007f95828c99b7 in tokio_timer::timer::{impl#4}::park (self=0x7f95625cb7d8) at /home/xzx/.cargo/git/checkouts/tokio-8e927faba632ed16/4394380/tokio-timer/src/timer/mod.rs:379 +#20 0x00007f95828a52c9 in tokio_timer::timer::Timer::turn (self=0x7f95625cb7d8, max_wait=...) at /home/xzx/.cargo/git/checkouts/tokio-8e927faba632ed16/4394380/tokio-timer/src/timer/mod.rs:256 +#21 0x00007f958289e969 in tikv_util::timer::start_timer_thread::{closure#0} () at components/tikv_util/src/timer.rs:259 +#22 0x00007f95828de0e9 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, ()> () at components/tikv_util/src/sys/thread.rs:441 +#23 0x00007f95828ad307 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#24 0x00007f95828b811d in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#25 0x00007f958285ef41 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#26 0x00007f958286a875 in std::panicking::try::do_call, ()>, ()>>, ()> (data=0x7f95625cb9e0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#27 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#28 0x00007f9582869a6f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#29 0x00007f95828b7711 in std::panic::catch_unwind, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#30 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#31 0x00007f95827edc4f in core::ops::function::FnOnce::call_once, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#32 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#33 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#34 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#35 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#36 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 19 (Thread 0x7f9562dff640 (LWP 2739549) "tso-worker"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f9581da7762 in futures_executor::local_pool::run_executor::{closure#0}<(), futures_executor::local_pool::block_on::{closure_env#0}, (kvproto::protos::pdpb::TsoRequest, grpcio::call::WriteFlags), pd_client::errors::Error>, futures_util::stream::try_stream::ErrInto, pd_client::errors::Error>>>> (thread_notify=0x7f9562dfb8d8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-executor-0.3.15/src/local_pool.rs:99 +#5 0x00007f9581d5f70b in std::thread::local::LocalKey>::try_with, futures_executor::local_pool::run_executor::{closure_env#0}<(), futures_executor::local_pool::block_on::{closure_env#0}, (kvproto::protos::pdpb::TsoRequest, grpcio::call::WriteFlags), pd_client::errors::Error>, futures_util::stream::try_stream::ErrInto, pd_client::errors::Error>>>>, ()> (self=0x7f9584008e30, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#6 0x00007f9581d5f5d4 in std::thread::local::LocalKey>::with, futures_executor::local_pool::run_executor::{closure_env#0}<(), futures_executor::local_pool::block_on::{closure_env#0}, (kvproto::protos::pdpb::TsoRequest, grpcio::call::WriteFlags), pd_client::errors::Error>, futures_util::stream::try_stream::ErrInto, pd_client::errors::Error>>>>, ()> (self=0x7f9584008e30, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 +#7 0x00007f9581da7129 in futures_executor::local_pool::run_executor<(), futures_executor::local_pool::block_on::{closure_env#0}, (kvproto::protos::pdpb::TsoRequest, grpcio::call::WriteFlags), pd_client::errors::Error>, futures_util::stream::try_stream::ErrInto, pd_client::errors::Error>>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-executor-0.3.15/src/local_pool.rs:86 +#8 0x00007f9581da78e6 in futures_executor::local_pool::block_on, (kvproto::protos::pdpb::TsoRequest, grpcio::call::WriteFlags), pd_client::errors::Error>, futures_util::stream::try_stream::ErrInto, pd_client::errors::Error>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-executor-0.3.15/src/local_pool.rs:315 +#9 0x00007f9581dfff48 in pd_client::tso::{impl#0}::new::{closure#0} () at components/pd_client/src/tso.rs:68 +#10 0x00007f9581e224e5 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0} () at components/tikv_util/src/sys/thread.rs:441 +#11 0x00007f9581dab447 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#12 0x00007f9581dac6c0 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#13 0x00007f9581e07944 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#14 0x00007f9581dea744 in std::panicking::try::do_call, ()>>, ()> (data=0x7f9562dcb560) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#15 0x00007f9581e0538b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#16 0x00007f9581dea6a8 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#17 0x00007f9581dac4e0 in std::panic::catch_unwind, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#18 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#19 0x00007f9581e0a43f in core::ops::function::FnOnce::call_once, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#20 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#22 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#23 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#24 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 18 (Thread 0x7f95639ff640 (LWP 2739548) "pdmonitor-0"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f95639fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f95639fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f95639fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279713916976, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f95639cb648) at src/pool/spawn.rs:305 +#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f95639cb648) at src/pool/worker.rs:36 +#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 +#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 +#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f95639cb860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 17 (Thread 0x7f95645ff640 (LWP 2739547) "grpc-server-4"): +#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 +#1 0x00007f9583904b84 in pollset_work(grpc_pollset*, grpc_pollset_worker**, long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958395ff8a in cq_next(grpc_completion_queue*, gpr_timespec, void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f95837ecbeb in grpcio::cq::CompletionQueue::next (self=0x7f95645cb758) at src/cq.rs:202 +#4 0x00007f95837d06c5 in grpcio::env::poll_queue (tx=...) at src/env.rs:20 +#5 0x00007f95837d14bc in grpcio::env::{impl#0}::build::{closure#0} () at src/env.rs:107 +#6 0x00007f95837ae4e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#7 0x00007f95837c4791 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#8 0x00007f95837c3451 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#9 0x00007f95837e29e1 in std::panicking::try::do_call>, ()> (data=0x7f95645cb9b0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#10 0x00007f95837eed2b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#11 0x00007f95837e295f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#12 0x00007f95837c45b1 in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#14 0x00007f95837e329f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 16 (Thread 0x7f9564dff640 (LWP 2739546) "grpc-server-3"): +#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 +#1 0x00007f9583904b84 in pollset_work(grpc_pollset*, grpc_pollset_worker**, long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958395ff8a in cq_next(grpc_completion_queue*, gpr_timespec, void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f95837ecbeb in grpcio::cq::CompletionQueue::next (self=0x7f9564dcb758) at src/cq.rs:202 +#4 0x00007f95837d06c5 in grpcio::env::poll_queue (tx=...) at src/env.rs:20 +#5 0x00007f95837d14bc in grpcio::env::{impl#0}::build::{closure#0} () at src/env.rs:107 +#6 0x00007f95837ae4e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#7 0x00007f95837c4791 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#8 0x00007f95837c3451 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#9 0x00007f95837e29e1 in std::panicking::try::do_call>, ()> (data=0x7f9564dcb9b0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#10 0x00007f95837eed2b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#11 0x00007f95837e295f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#12 0x00007f95837c45b1 in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#14 0x00007f95837e329f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 15 (Thread 0x7f95657ff640 (LWP 2739545) "grpc-server-2"): +#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 +#1 0x00007f9583904b84 in pollset_work(grpc_pollset*, grpc_pollset_worker**, long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958395ff8a in cq_next(grpc_completion_queue*, gpr_timespec, void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f95837ecbeb in grpcio::cq::CompletionQueue::next (self=0x7f95657cb758) at src/cq.rs:202 +#4 0x00007f95837d06c5 in grpcio::env::poll_queue (tx=...) at src/env.rs:20 +#5 0x00007f95837d14bc in grpcio::env::{impl#0}::build::{closure#0} () at src/env.rs:107 +#6 0x00007f95837ae4e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#7 0x00007f95837c4791 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#8 0x00007f95837c3451 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#9 0x00007f95837e29e1 in std::panicking::try::do_call>, ()> (data=0x7f95657cb9b0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#10 0x00007f95837eed2b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#11 0x00007f95837e295f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#12 0x00007f95837c45b1 in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#14 0x00007f95837e329f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 14 (Thread 0x7f9565dfb640 (LWP 2739544) "grpc-server-1"): +#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 +#1 0x00007f9583904b84 in pollset_work(grpc_pollset*, grpc_pollset_worker**, long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958395ff8a in cq_next(grpc_completion_queue*, gpr_timespec, void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f95837ecbeb in grpcio::cq::CompletionQueue::next (self=0x7f9565dc7758) at src/cq.rs:202 +#4 0x00007f95837d06c5 in grpcio::env::poll_queue (tx=...) at src/env.rs:20 +#5 0x00007f95837d14bc in grpcio::env::{impl#0}::build::{closure#0} () at src/env.rs:107 +#6 0x00007f95837ae4e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#7 0x00007f95837c4791 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#8 0x00007f95837c3451 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#9 0x00007f95837e29e1 in std::panicking::try::do_call>, ()> (data=0x7f9565dc79b0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#10 0x00007f95837eed2b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#11 0x00007f95837e295f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#12 0x00007f95837c45b1 in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#14 0x00007f95837e329f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 13 (Thread 0x7f9565ffc640 (LWP 2739543) "grpc-server-0"): +#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 +#1 0x00007f9583904b84 in pollset_work(grpc_pollset*, grpc_pollset_worker**, long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958395ff8a in cq_next(grpc_completion_queue*, gpr_timespec, void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f95837ecbeb in grpcio::cq::CompletionQueue::next (self=0x7f9565fc8758) at src/cq.rs:202 +#4 0x00007f95837d06c5 in grpcio::env::poll_queue (tx=...) at src/env.rs:20 +#5 0x00007f95837d14bc in grpcio::env::{impl#0}::build::{closure#0} () at src/env.rs:107 +#6 0x00007f95837ae4e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#7 0x00007f95837c4791 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#8 0x00007f95837c3451 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#9 0x00007f95837e29e1 in std::panicking::try::do_call>, ()> (data=0x7f9565fc89b0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#10 0x00007f95837eed2b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#11 0x00007f95837e295f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#12 0x00007f95837c45b1 in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#14 0x00007f95837e329f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 12 (Thread 0x7f95669fd640 (LWP 2739542) "grpc_global_tim"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f958386d4a9 in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9583867500 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f9583917710 in timer_thread(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 11 (Thread 0x7f95673fe640 (LWP 2739541) "resolver-execut"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f958386d4a9 in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9583867500 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f9583909ab3 in grpc_core::Executor::ThreadMain(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 10 (Thread 0x7f9567dff640 (LWP 2739540) "default-executo"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f958386d4a9 in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#4 0x00007f9583867500 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#5 0x00007f9583909ab3 in grpc_core::Executor::ThreadMain(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 9 (Thread 0x7f95685ff640 (LWP 2739539) "time-monitor"): +#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 +#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 +#2 0x00007f9583cea820 in std::sys::unix::thread::Thread::sleep () at library/std/src/sys/unix/thread.rs:243 +#3 std::thread::sleep () at library/std/src/thread/mod.rs:870 +#4 0x00007f958281b6ab in tikv_util::time::{impl#4}::new::{closure#0} std::time::SystemTime> () at components/tikv_util/src/time.rs:174 +#5 0x00007f95828ddfbe in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0} std::time::SystemTime>, ()> () at components/tikv_util/src/sys/thread.rs:441 +#6 0x00007f95828ad317 in std::sys_common::backtrace::__rust_begin_short_backtrace std::time::SystemTime>, ()>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#7 0x00007f95828b8186 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} std::time::SystemTime>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#8 0x00007f958285edb9 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0} std::time::SystemTime>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#9 0x00007f958286a97d in std::panicking::try::do_call std::time::SystemTime>, ()>, ()>>, ()> (data=0x7f95685cba50) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#10 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#11 0x00007f9582869b9e in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe std::time::SystemTime>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#12 0x00007f95828b6eaf in std::panic::catch_unwind std::time::SystemTime>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} std::time::SystemTime>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#14 0x00007f95827ed7ef in core::ops::function::FnOnce::call_once std::time::SystemTime>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 8 (Thread 0x7f9568d7f640 (LWP 2739538) "slogger"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 +#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 +#3 std::thread::park () at library/std/src/thread/mod.rs:1066 +#4 0x00007f9582826c0b in crossbeam_channel::context::Context::wait_until (self=0x7f9568d491a8, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 +#5 0x00007f95828287c7 in crossbeam_channel::flavors::array::{impl#1}::recv::{closure#1} (cx=0x7f9568d491a8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/array.rs:432 +#6 0x00007f95827e5612 in crossbeam_channel::context::{impl#0}::with::{closure#0}, ()> (cx=0x7f9568d491a8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 +#7 0x00007f95827e5563 in crossbeam_channel::context::{impl#0}::with::{closure#1}, ()> (cell=0x7f9568d7b798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 +#8 0x00007f95828b19cb in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, ()>, ()> (self=0x7f958405cd10, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 +#9 0x00007f95827e527e in crossbeam_channel::context::Context::with, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 +#10 0x00007f95828286e5 in crossbeam_channel::flavors::array::Channel::recv (self=0x7f956a809c80, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/array.rs:421 +#11 0x00007f95828266ea in crossbeam_channel::channel::Receiver::recv (self=0x7f9568d496b8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:814 +#12 0x00007f957dfa3cff in slog_async::{impl#5}::spawn_thread::{closure#0}>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>> () at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/slog-async-2.6.0/lib.rs:286 +#13 0x00007f957e2c2927 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#14 0x00007f957e4446f4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#15 0x00007f957e258494 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#16 0x00007f957e2c60e0 in std::panicking::try::do_call>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()>>, ()> (data=0x7f9568d4a1e0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#17 0x00007f957e40507b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#18 0x00007f957e2c3868 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#19 0x00007f957e444085 in std::panic::catch_unwind>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#20 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#21 0x00007f957e146bcf in core::ops::function::FnOnce::call_once>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#22 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#23 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#24 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#25 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#26 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 7 (Thread 0x7f95697ff640 (LWP 2739537) "archive-worker-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f95697fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f95697fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f95697fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279713917136, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f95697cac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f95697cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f95697cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 6 (Thread 0x7f9569fff640 (LWP 2739536) "archive-worker-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9569ffb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9569ffb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9569ffb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279713915216, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9569fcac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f9569fcab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f9569fcb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 5 (Thread 0x7f956a7ff640 (LWP 2739535) "archive-worker-"): +#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 +#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f956a7fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 +#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f956a7fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 +#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f956a7fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 +#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 +#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279713915056, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 +#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f956a7cac70) at src/pool/spawn.rs:305 +#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f956a7cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 +#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 +#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 +#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 +#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 +#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 +#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f956a7cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 +#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so +#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 +#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 +#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 +#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 +#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 +#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 +#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 4 (Thread 0x7f956c1fe640 (LWP 2739532) "RaftStoreProxy"): +#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 +#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 +#2 0x00007f9583cea820 in std::sys::unix::thread::Thread::sleep () at library/std/src/sys/unix/thread.rs:243 +#3 std::thread::sleep () at library/std/src/thread/mod.rs:870 +#4 0x00007f957c39f4b3 in proxy_server::run::run_impl (config=..., proxy_config=..., engine_store_server_helper=0x7ffd1e7b8210) at proxy_components/proxy_server/src/run.rs:297 +#5 0x00007f957c3b3dc2 in proxy_server::run::run_tikv_proxy (config=..., proxy_config=..., engine_store_server_helper=0x7ffd1e7b8210) at proxy_components/proxy_server/src/run.rs:441 +#6 0x00007f957d4f6b4d in proxy_server::proxy::run_proxy (argc=15, argv=0x7f956a801000, engine_store_server_helper=0x7ffd1e7b8210) at proxy_components/proxy_server/src/proxy.rs:332 +#7 0x00007f957a6280e8 in raftstore_proxy::run_raftstore_proxy_ffi (argc=15, argv=0x7f956a801000, helper=0x7ffd1e7b8210) at raftstore-proxy/src/lib.rs:19 +#8 0x0000558c1ab7865c in DB::RaftStoreProxyRunner::runRaftStoreProxyFFI (pv=0x7f956f84ef10) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/ProxyStateMachine.h:244 +#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 3 (Thread 0x7f956cbff640 (LWP 2739500) "jemalloc_bg_thd"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f957068996f in pthread_cond_timedwait@@GLIBC_2.3.2 () from /lib64/libc.so.6 +#2 0x0000558c275f88b3 in background_thread_sleep (tsdn=0x7f956cbfba90, info=0x7f9570417480, interval=5000000000) at /DATA/disk3/xzx/tiflash/contrib/jemalloc/src/background_thread.c:159 +#3 0x0000558c275f841c in background_work_sleep_once (tsdn=0x7f956cbfba90, info=0x7f9570417480, ind=0) at /DATA/disk3/xzx/tiflash/contrib/jemalloc/src/background_thread.c:229 +#4 0x0000558c275f8018 in background_thread0_work (tsd=0x7f956cbfba90) at /DATA/disk3/xzx/tiflash/contrib/jemalloc/src/background_thread.c:374 +#5 0x0000558c275f7d9f in background_work (tsd=0x7f956cbfba90, ind=0) at /DATA/disk3/xzx/tiflash/contrib/jemalloc/src/background_thread.c:412 +#6 0x0000558c275f77ce in background_thread_entry (ind_arg=0x0) at /DATA/disk3/xzx/tiflash/contrib/jemalloc/src/background_thread.c:444 +#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 2 (Thread 0x7f956e5ff640 (LWP 2739499) "SignalListener"): +#0 0x00007f95706fdfcc in read () from /lib64/libc.so.6 +#1 0x0000558c1ac54b52 in DB::ReadBufferFromFileDescriptor::nextImpl (this=0x7f956e5cb290) at /DATA/disk3/xzx/tiflash/dbms/src/IO/Buffer/ReadBufferFromFileDescriptor.cpp:64 +#2 0x0000558c1aaa475f in DB::ReadBuffer::next (this=0x7f956e5cb290) at /DATA/disk3/xzx/tiflash/dbms/src/IO/Buffer/ReadBuffer.h:75 +#3 0x0000558c23425538 in DB::ReadBuffer::eof (this=0x7f956e5cb290) at /DATA/disk3/xzx/tiflash/dbms/src/IO/Buffer/ReadBuffer.h:100 +#4 SignalListener::run (this=0x7f95702f2ae0) at /DATA/disk3/xzx/tiflash/libs/libdaemon/src/BaseDaemon.cpp:294 +#5 0x0000558c27bf81ea in Poco::(anonymous namespace)::RunnableHolder::run (this=0x7f957020ef10) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread.cpp:43 +#6 0x0000558c27bf72ea in Poco::ThreadImpl::runnableEntry (pThread=0x7ffd1e7b9e58) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread_STD.cpp:139 +#7 0x0000558c27bf9760 in std::__1::__invoke[abi:ue170006](void* (*&&)(void*), Poco::ThreadImpl*&&) (__f=@0x7f95702f2b28: 0x558c27bf7280 , __args=@0x7f95702f2b30: 0x7ffd1e7b9e58) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 +#8 0x0000558c27bf972e in std::__1::__thread_execute[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*, 2ul>(std::__1::tuple >, void* (*)(void*), Poco::ThreadImpl*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 +#9 0x0000558c27bf9512 in std::__1::__thread_proxy[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*> >(void*) (__vp=0x7f95702f2b20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 +#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 +#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 +Thread 1 (Thread 0x7f95845be900 (LWP 2739406) "TiFlashMain"): +#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 +#1 0x00007f957068bca4 in __pthread_clockjoin_ex () from /lib64/libc.so.6 +#2 0x00007f95847adbb3 in std::__1::thread::join() () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 +#3 0x0000558c259b53fc in DB::TaskThreadPool::waitForStop (this=0x7f91cfae6dc0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp:53 +#4 0x0000558c25988d5d in DB::TaskScheduler::~TaskScheduler (this=0x7f91cfae6d80) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/TaskScheduler.cpp:38 +#5 0x0000558c1ab8d2bb in std::__1::default_delete::operator()[abi:ue170006](DB::TaskScheduler*) const (this=0x558c29257c38 , __ptr=0x7f91cfae6d80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__memory/unique_ptr.h:68 +#6 0x0000558c1ab8d1dc in std::__1::unique_ptr >::reset[abi:ue170006](DB::TaskScheduler*) (this=0x558c29257c38 , __p=0x0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__memory/unique_ptr.h:300 +#7 0x0000558c1ab5c549 in DB::Server::main(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&)::$_29::operator()() const (this=0x7ffd1e7b5ef0) at /DATA/disk3/xzx/tiflash/dbms/src/Server/Server.cpp:1207 +#8 0x0000558c1ab591c5 in ext::ScopeGuard, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&)::$_29>::~ScopeGuard() (this=0x7ffd1e7b5ef0) at /DATA/disk3/xzx/tiflash/libs/libcommon/include/ext/scope_guard.h:33 +#9 0x0000558c1ab5896f in DB::Server::main (this=0x7ffd1e7b9d08) at /DATA/disk3/xzx/tiflash/dbms/src/Server/Server.cpp:1260 +#10 0x0000558c27a41fc5 in Poco::Util::Application::run (this=0x7ffd1e7b9d08) at /DATA/disk3/xzx/tiflash/contrib/poco/Util/src/Application.cpp:335 +#11 0x0000558c27a60205 in Poco::Util::ServerApplication::run (this=0x7ffd1e7b9d08) at /DATA/disk3/xzx/tiflash/contrib/poco/Util/src/ServerApplication.cpp:94 +#12 0x0000558c1ab4cbc7 in DB::Server::run (this=0x7ffd1e7b9d08) at /DATA/disk3/xzx/tiflash/dbms/src/Server/Server.cpp:179 +#13 0x0000558c27a603e0 in Poco::Util::ServerApplication::run (this=0x7ffd1e7b9d08, argc=3, pArgv=0x7f95702f2980) at /DATA/disk3/xzx/tiflash/contrib/poco/Util/src/ServerApplication.cpp:618 +#14 0x0000558c1ab59338 in mainEntryClickHouseServer (argc=3, argv=0x7f95702f2980) at /DATA/disk3/xzx/tiflash/dbms/src/Server/Server.cpp:1280 +#15 0x0000558c1aa031a1 in main (argc_=4, argv_=0x7ffd1e7ba2a8) at /DATA/disk3/xzx/tiflash/dbms/src/Server/main.cpp:172 From f443e8e60bd7f4cf3ca742ef7bd2a62b78111dbf Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 9 Jul 2025 16:25:49 +0800 Subject: [PATCH 090/118] remove --- stack | 32007 -------------------------------------------------------- 1 file changed, 32007 deletions(-) delete mode 100644 stack diff --git a/stack b/stack deleted file mode 100644 index d00cf80266f..00000000000 --- a/stack +++ /dev/null @@ -1,32007 +0,0 @@ -Thread 1351 (Thread 0x7f903488f640 (LWP 2743138) "default-executo"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x0000558c288f9efe in absl::lts_20211102::synchronization_internal::FutexImpl::WaitUntil (v=0x7f9575a67340, val=0, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/futex.h:104 -#2 0x0000558c288f9bfb in absl::lts_20211102::synchronization_internal::Waiter::Wait (this=0x7f9575a67340, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/waiter.cc:95 -#3 0x0000558c288f9816 in AbslInternalPerThreadSemWait_lts_20211102 (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.cc:93 -#4 0x0000558c288f6bbd in absl::lts_20211102::synchronization_internal::PerThreadSem::Wait (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.h:112 -#5 0x0000558c288efd56 in absl::lts_20211102::Mutex::DecrementSynchSem (mu=0x7f9493fa3060, w=0x7f9575a67300, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:579 -#6 0x0000558c288f4e7a in absl::lts_20211102::CondVar::WaitCommon (this=0x7f9493fa3078, mutex=0x7f9493fa3060, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2559 -#7 0x0000558c288f50aa in absl::lts_20211102::CondVar::Wait (this=0x7f9493fa3078, mu=0x7f9493fa3060) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2594 -#8 0x0000558c2832ff8b in gpr_cv_wait (cv=0x7f9493fa3078, mu=0x7f9493fa3060, abs_deadline=...) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gpr/sync_abseil.cc:85 -#9 0x0000558c27df194f in grpc_core::Executor::ThreadMain (arg=0x7f9493fa3060) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/executor.cc:238 -#10 0x0000558c28339f22 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::operator()(void*) const (this=0x7f903485bc87, v=0x7f938f4e5d50) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:140 -#11 0x0000558c28339e09 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) (v=0x7f938f4e5d50) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:110 -#12 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#13 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1350 (Thread 0x7f903e99e640 (LWP 2743051) "default-executo"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f958386d4a9 in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9583867500 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f9583909ab3 in grpc_core::Executor::ThreadMain(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1349 (Thread 0x7f903f39f640 (LWP 2743046) "default-executo"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f958386d4a9 in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9583867500 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f9583909ab3 in grpc_core::Executor::ThreadMain(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1348 (Thread 0x7f91984aa640 (LWP 2741765) "io_pool"): -#0 0x0000558c276cdd16 in LZ4_count (pIn=0x7f8fd4d49d08 "", pMatch=0x7f8fd4d49d04 "", pInLimit=0x7f8fd4df6882 "") at /DATA/disk3/xzx/tiflash/contrib/lz4/lib/lz4.c:674 -#1 LZ4_compress_generic_validated (cctx=0x7f9198470510, source=0x7f8fd4d0f740 "\004", dest=0x7f8fd408f589 "O\004", inputSize=946503, inputConsumed=0x0, maxOutputSize=0, outputDirective=notLimited, tableType=byU32, dictDirective=noDict, dictIssue=noDictIssue, acceleration=1) at /DATA/disk3/xzx/tiflash/contrib/lz4/lib/lz4.c:1153 -#2 LZ4_compress_generic (cctx=0x7f9198470510, src=0x7f8fd4d0f740 "\004", dst=0x7f8fd408f589 "O\004", srcSize=946503, inputConsumed=0x0, dstCapacity=0, outputDirective=notLimited, tableType=byU32, dictDirective=noDict, dictIssue=noDictIssue, acceleration=1) at /DATA/disk3/xzx/tiflash/contrib/lz4/lib/lz4.c:1339 -#3 LZ4_compress_fast_extState (state=0x7f9198470510, source=0x7f8fd4d0f740 "\004", dest=0x7f8fd408f589 "O\004", inputSize=946503, maxOutputSize=950230, acceleration=1) at /DATA/disk3/xzx/tiflash/contrib/lz4/lib/lz4.c:1357 -#4 0x0000558c276eb34a in LZ4_compress_fast (source=0x7f8fd4d0f740 "\004", dest=0x7f8fd408f589 "O\004", inputSize=946503, maxOutputSize=950230, acceleration=1) at /DATA/disk3/xzx/tiflash/contrib/lz4/lib/lz4.c:1426 -#5 0x0000558c235cc176 in DB::CompressionCodecLZ4::doCompressData (this=0x7f94ce80b258, source=0x7f8fd4d0f740 "\004", source_size=946503, dest=0x7f8fd408f589 "O\004") at /DATA/disk3/xzx/tiflash/dbms/src/IO/Compression/CompressionCodecLZ4.cpp:46 -#6 0x0000558c235ef8d2 in DB::ICompressionCodec::compress (this=0x7f94ce80b258, source=0x7f8fd4d0f740 "\004", source_size=946503, dest=0x7f8fd408f580 "\202*M\003") at /DATA/disk3/xzx/tiflash/dbms/src/IO/Compression/ICompressionCodec.cpp:38 -#7 0x0000558c235ac7d9 in DB::CompressedWriteBuffer::nextImpl (this=0x7f94bb518070) at /DATA/disk3/xzx/tiflash/dbms/src/IO/Compression/CompressedWriteBuffer.cpp:34 -#8 0x0000558c1aa06234 in DB::WriteBuffer::next (this=0x7f94bb518070) at /DATA/disk3/xzx/tiflash/dbms/src/IO/Buffer/WriteBuffer.h:59 -#9 0x0000558c2347e179 in DB::NativeBlockOutputStream::flush (this=0x7f94bb51e000) at /DATA/disk3/xzx/tiflash/dbms/src/DataStreams/NativeBlockOutputStream.cpp:56 -#10 0x0000558c23d34881 in DB::SpillHandler::SpillWriter::finishWrite (this=0x7f94bb518000) at /DATA/disk3/xzx/tiflash/dbms/src/Core/SpillHandler.cpp:57 -#11 0x0000558c23d35c66 in DB::SpillHandler::spillBlocks (this=0x7f91984759b0, blocks=...) at /DATA/disk3/xzx/tiflash/dbms/src/Core/SpillHandler.cpp:162 -#12 0x0000558c23d21c0a in DB::Spiller::spillBlocks (this=0x7f94bb513280, blocks=..., partition_id=0) at /DATA/disk3/xzx/tiflash/dbms/src/Core/Spiller.cpp:226 -#13 0x0000558c25c24f65 in DB::CTEPartition::spillBlocks (this=0x7f93a4cc8000) at /DATA/disk3/xzx/tiflash/dbms/src/Operators/CTEPartition.cpp:146 -#14 0x0000558c25c20471 in DB::CTE::spillBlocks (this=0x7f93a4cd2558, partition_id=0) at /DATA/disk3/xzx/tiflash/dbms/src/Operators/CTE.cpp:94 -#15 0x0000558c25c310ad in DB::CTESinkOp::executeIOImpl (this=0x7f94ac25df00) at /DATA/disk3/xzx/tiflash/dbms/src/Operators/CTESinkOp.cpp:61 -#16 0x0000558c259823be in DB::Operator::executeIO (this=0x7f94ac25df00) at /DATA/disk3/xzx/tiflash/dbms/src/Operators/Operator.cpp:81 -#17 0x0000558c2598120e in DB::PipelineExec::executeIOImpl (this=0x7f93a3cc5230) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp:194 -#18 0x0000558c2597df15 in DB::PipelineExec::executeIO (this=0x7f93a3cc5230) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp:181 -#19 0x0000558c2599a1cf in DB::PipelineTaskBase::runExecuteIO (this=0x7f94ac25e0a0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/Tasks/Impls/PipelineTaskBase.h:92 -#20 0x0000558c25999b3c in DB::PipelineTask::executeIOImpl (this=0x7f94ac25dfe0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/Tasks/Impls/PipelineTask.h:40 -#21 0x0000558c259a99c8 in DB::Task::executeIO (this=0x7f94ac25dfe0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp:140 -#22 0x0000558c259b60ed in DB::IOImpl::exec (task=...) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPoolImpl.h:48 -#23 0x0000558c259b5e29 in DB::TaskThreadPool::handleTask (this=0x7f91cfae6dc0, task=...) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp:105 -#24 0x0000558c259b5a0e in DB::TaskThreadPool::doLoop (this=0x7f91cfae6dc0, thread_no=0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp:84 -#25 0x0000558c259b5261 in DB::TaskThreadPool::loop (this=0x7f91cfae6dc0, thread_no=0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp:63 -#26 0x0000558c259b71a4 in std::__1::__invoke[abi:ue170006]::*)(unsigned long), DB::TaskThreadPool*, unsigned long, void>(void (DB::TaskThreadPool::*&&)(unsigned long), DB::TaskThreadPool*&&, unsigned long&&) (__f=@0x7f91cf72b358: (void (DB::TaskThreadPool::*)(DB::TaskThreadPool * const, unsigned long)) 0x558c259b5220 ::loop(unsigned long)>, __a0=@0x7f91cf72b368: 0x7f91cfae6dc0, __args=@0x7f91cf72b370: 0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#27 0x0000558c259b711f in std::__1::__thread_execute[abi:ue170006] >, void (DB::TaskThreadPool::*)(unsigned long), DB::TaskThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, void (DB::TaskThreadPool::*)(unsigned long), DB::TaskThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#28 0x0000558c259b6ed2 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::TaskThreadPool::*)(unsigned long), DB::TaskThreadPool*, unsigned long> >(void*) (__vp=0x7f91cf72b350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1347 (Thread 0x7f91c5ef3640 (LWP 2741688) "TiFlashMain"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09f968, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f956f09f968, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c268e60b9 in std::__1::condition_variable::wait_until >, DB::MockLocalAdmissionController::refillTokenBucket()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::MockLocalAdmissionController::refillTokenBucket()::$_0) (this=0x7f956f09f968, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c268e5ddf in std::__1::condition_variable::wait_for[abi:ue170006], DB::MockLocalAdmissionController::refillTokenBucket()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::MockLocalAdmissionController::refillTokenBucket()::$_0) (this=0x7f956f09f968, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 -#6 0x0000558c268e5aec in DB::MockLocalAdmissionController::refillTokenBucket (this=0x7f956f09f940) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/ResourceControl/MockLocalAdmissionController.cpp:25 -#7 0x0000558c1ab8c608 in DB::MockLocalAdmissionController::MockLocalAdmissionController()::{lambda()#1}::operator()() const (this=0x7f91cf5e7508) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/ResourceControl/MockLocalAdmissionController.h:45 -#8 0x0000558c1ab8c5c5 in std::__1::__invoke[abi:ue170006](DB::MockLocalAdmissionController::MockLocalAdmissionController()::{lambda()#1}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#9 0x0000558c1ab8c59d in std::__1::__thread_execute[abi:ue170006] >, DB::MockLocalAdmissionController::MockLocalAdmissionController()::{lambda()#1}>(std::__1::tuple >, DB::MockLocalAdmissionController::MockLocalAdmissionController()::{lambda()#1}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#10 0x0000558c1ab8c3d2 in std::__1::__thread_proxy[abi:ue170006] >, DB::MockLocalAdmissionController::MockLocalAdmissionController()::{lambda()#1}> >(void*) (__vp=0x7f91cf5e7500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#11 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#12 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1346 (Thread 0x7f91c85f4640 (LWP 2741348) "ReadIndexWkr-0"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09fb40, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f956f09fb40, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c26b22249 in DB::AsyncWaker::Notifier::blockedWaitUtil (this=0x7f956f09fac0, time_point=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/Read/AsyncNotifier.cpp:30 -#5 0x0000558c237572f0 in DB::AsyncNotifier::blockedWaitFor (this=0x7f956f09fac0, duration=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/Utils.h:97 -#6 0x0000558c26b71299 in DB::ReadIndexWorkerManager::ReadIndexRunner::blockedWaitFor (this=0x7f9493f3e400, timeout=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/Read/ReadIndexWorkerManager.cpp:140 -#7 0x0000558c26b72450 in DB::ReadIndexWorkerManager::ReadIndexRunner::asyncRun()::$_0::operator()() const (this=0x7f94936ee698) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/Read/ReadIndexWorkerManager.cpp:166 -#8 0x0000558c26b71e05 in std::__1::__invoke[abi:ue170006](DB::ReadIndexWorkerManager::ReadIndexRunner::asyncRun()::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#9 0x0000558c26b71ddd in std::__1::__thread_execute[abi:ue170006] >, DB::ReadIndexWorkerManager::ReadIndexRunner::asyncRun()::$_0>(std::__1::tuple >, DB::ReadIndexWorkerManager::ReadIndexRunner::asyncRun()::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#10 0x0000558c26b71c12 in std::__1::__thread_proxy[abi:ue170006] >, DB::ReadIndexWorkerManager::ReadIndexRunner::asyncRun()::$_0> >(void*) (__vp=0x7f94936ee690) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#11 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#12 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1345 (Thread 0x7f91c87f5640 (LWP 2741347) "RaftStoreProxy"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957dfc02bb in crossbeam_channel::context::Context::wait_until (self=0x7f91c87c14d8, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957e132377 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1} (cx=0x7f91c87c14d8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957dfc0c52 in crossbeam_channel::context::{impl#0}::with::{closure#0}, ()> (cx=0x7f91c87c14d8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957dfc0ba3 in crossbeam_channel::context::{impl#0}::with::{closure#1}, ()> (cell=0x7f91c87f1798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957e43dcab in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, ()>, ()> (self=0x7f9583d91288, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957dfc050e in crossbeam_channel::context::Context::with, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957e13214b in crossbeam_channel::flavors::list::Channel::recv (self=0x7f956a8c7e00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957dfbfaee in crossbeam_channel::channel::Receiver::recv (self=0x7f91c87c1980) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957e3fff9b in tikv_util::mpsc::Receiver::recv (self=0x7f91c87c1980) at components/tikv_util/src/mpsc/mod.rs:153 -#13 0x00007f957c3ab81a in proxy_server::run::run_impl::{closure#0} () at proxy_components/proxy_server/src/run.rs:269 -#14 0x00007f957a80ae27 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#15 0x00007f957cb7a7f6 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#16 0x00007f957c9e6769 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#17 0x00007f957bdf3add in std::panicking::try::do_call, ()>>, ()> (data=0x7f91c87c1a50) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#18 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#19 0x00007f957bdd9b8e in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#20 0x00007f957cb78daf in std::panic::catch_unwind, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#21 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#22 0x00007f957d67f3bf in core::ops::function::FnOnce::call_once, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#23 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#24 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#25 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#26 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#27 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1344 (Thread 0x7f91c89f6640 (LWP 2741346) "status-server"): -#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 -#1 0x00007f9583185778 in mio::sys::unix::selector::epoll::Selector::select (self=0x7f91ea7113b8, events=0x7f91ea711398, timeout=...) at src/sys/unix/selector/epoll.rs:97 -#2 0x00007f9583184fd1 in mio::poll::Poll::poll (self=0x7f91ea7113b0, events=0x7f91ea711398, timeout=...) at src/poll.rs:426 -#3 0x00007f9583167b7c in tokio::runtime::io::driver::Driver::turn (self=0x7f91ea711398, handle=0x7f91ea71e118, max_wait=...) at src/runtime/io/driver.rs:149 -#4 0x00007f95831677ee in tokio::runtime::io::driver::Driver::park (self=0x7f91ea711398, rt_handle=0x7f91ea71e118) at src/runtime/io/driver.rs:122 -#5 0x00007f958316a887 in tokio::runtime::signal::Driver::park (self=0x7f91ea711398, handle=0x7f91ea71e118) at src/runtime/signal/mod.rs:92 -#6 0x00007f9583152787 in tokio::runtime::process::Driver::park (self=0x7f91ea711398, handle=0x7f91ea71e118) at src/runtime/process.rs:32 -#7 0x00007f95830f1aca in tokio::runtime::driver::IoStack::park (self=0x7f91ea711398, handle=0x7f91ea71e118) at src/runtime/driver.rs:175 -#8 0x00007f958316a02e in tokio::runtime::time::Driver::park_internal (self=0x7f91ea711398, rt_handle=0x7f91ea71e118, limit=...) at src/runtime/time/mod.rs:235 -#9 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f91ea711398, handle=0x7f91ea71e118) at src/runtime/time/mod.rs:165 -#10 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f91ea711390, handle=0x7f91ea71e118) at src/runtime/driver.rs:332 -#11 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f91ea711390, handle=0x7f91ea71e118) at src/runtime/driver.rs:71 -#12 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f91ea679620, driver=0x7f91ea711390, handle=0x7f91ea71e118) at src/runtime/scheduler/multi_thread/park.rs:184 -#13 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea679620, handle=0x7f91ea71e118) at src/runtime/scheduler/multi_thread/park.rs:117 -#14 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c89c19a0, handle=0x7f91ea71e118) at src/runtime/scheduler/multi_thread/park.rs:67 -#15 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c89c1e98, core=0x7f956917b940, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#16 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c89c1e98, core=0x7f956917b940) at src/runtime/scheduler/multi_thread/worker.rs:702 -#17 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c89c1e98, core=0x7f956917b940) at src/runtime/scheduler/multi_thread/worker.rs:553 -#18 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#19 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c89c3610, t=0x7f91c89c1e90, f=...) at src/runtime/context/scoped.rs:40 -#20 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c89c35d8) at src/runtime/context.rs:180 -#21 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#22 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#23 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c89c1e90, f=...) at src/runtime/context.rs:180 -#24 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#25 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c89c2098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#26 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#27 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#28 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c89c2220) at src/runtime/blocking/task.rs:42 -#29 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea7114a8) at src/runtime/task/core.rs:328 -#30 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea7114a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#31 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea7114a0, cx=...) at src/runtime/task/core.rs:317 -#32 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#33 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#34 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c89c2398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#35 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#36 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#37 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#38 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea7114a0, cx=...) at src/runtime/task/harness.rs:473 -#39 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c89c25b0) at src/runtime/task/harness.rs:208 -#40 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#41 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#42 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#43 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#44 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#45 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132ded10, worker_thread_id=1) at src/runtime/blocking/pool.rs:513 -#46 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#47 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#48 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#49 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#50 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c89c2a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#51 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#52 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#53 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#54 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#55 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#56 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#57 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#58 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#59 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#60 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1343 (Thread 0x7f91c8bf7640 (LWP 2741345) "status-server"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c8bf3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c8bf3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c8bf3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264679642576, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea6795d0, mutex=0x7f91ea6795d8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea6795d0, mutex_guard=0x7f91c8bc2628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea6795d0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea6795c0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea6795c0, handle=0x7f91ea71e118) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c8bc29a0, handle=0x7f91ea71e118) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c8bc2e98, core=0x7f956917b8f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c8bc2e98, core=0x7f956917b8f0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c8bc2e98, core=0x7f956917b8f0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c8bc4610, t=0x7f91c8bc2e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c8bc45d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c8bc2e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c8bc3098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c8bc3220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea711428) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea711428, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea711420, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c8bc3398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea711420, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c8bc35b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132ded10, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c8bc3a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1342 (Thread 0x7f91c9078640 (LWP 2741344) "snap-sender"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9074750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9074750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9074730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264675270480, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea24df50, mutex=0x7f91ea24df58, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea24df50, mutex_guard=0x7f91c9043628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea24df50, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea24df40) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea24df40, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c90439a0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9043e98, core=0x7f956917b350, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9043e98, core=0x7f956917b350) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9043e98, core=0x7f956917b350) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9045610, t=0x7f91c9043e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c90455d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9043e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9044098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9044220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea314128) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea314128, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea314120, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9044398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea314120, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c90445b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f9247e09150, worker_thread_id=3) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9044a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1341 (Thread 0x7f91c9279640 (LWP 2741343) "snap-sender"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9275750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9275750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9275730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264675270672, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea24e010, mutex=0x7f91ea24e018, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea24e010, mutex_guard=0x7f91c9244628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea24e010, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea24e000) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea24e000, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c92449a0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9244e98, core=0x7f956917b3f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9244e98, core=0x7f956917b3f0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9244e98, core=0x7f956917b3f0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9246610, t=0x7f91c9244e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c92465d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9244e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9245098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9245220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea314228) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea314228, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea314220, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9245398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea314220, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c92455b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f9247e09150, worker_thread_id=2) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9245a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1340 (Thread 0x7f91c947a640 (LWP 2741342) "snap-sender"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9476750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9476750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9476730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264675270576, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea24dfb0, mutex=0x7f91ea24dfb8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea24dfb0, mutex_guard=0x7f91c9445628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea24dfb0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea24dfa0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea24dfa0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c94459a0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9445e98, core=0x7f956917b3a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9445e98, core=0x7f956917b3a0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9445e98, core=0x7f956917b3a0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9447610, t=0x7f91c9445e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c94475d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9445e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9446098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9446220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea3141a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea3141a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea3141a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9446398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea3141a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c94465b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f9247e09150, worker_thread_id=1) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9446a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1339 (Thread 0x7f91c967b640 (LWP 2741341) "snap-sender"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9677750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9677750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9677730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264675270232, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea24de58, mutex=0x7f91ea24de60, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea24de58, mutex_guard=0x7f91c9646368) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea24de58, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830ef2f6 in tokio::runtime::park::Inner::park (self=0x7f91ea24de50) at src/runtime/park.rs:116 -#10 0x00007f95830ef113 in tokio::runtime::park::ParkThread::park (self=0x7f91ea314020) at src/runtime/park.rs:62 -#11 0x00007f95830f1adf in tokio::runtime::driver::IoStack::park (self=0x7f91ea314018, handle=0x7f91ea093d18) at src/runtime/driver.rs:176 -#12 0x00007f95830f2090 in tokio::runtime::driver::TimeDriver::park (self=0x7f91ea314010, handle=0x7f91ea093d18) at src/runtime/driver.rs:333 -#13 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f91ea314010, handle=0x7f91ea093d18) at src/runtime/driver.rs:71 -#14 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f91ea24dee0, driver=0x7f91ea314010, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:184 -#15 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea24dee0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:117 -#16 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c96469a0, handle=0x7f91ea093d18) at src/runtime/scheduler/multi_thread/park.rs:67 -#17 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9646e98, core=0x7f956917b300, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#18 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9646e98, core=0x7f956917b300) at src/runtime/scheduler/multi_thread/worker.rs:702 -#19 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9646e98, core=0x7f956917b300) at src/runtime/scheduler/multi_thread/worker.rs:553 -#20 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#21 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9648610, t=0x7f91c9646e90, f=...) at src/runtime/context/scoped.rs:40 -#22 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c96485d8) at src/runtime/context.rs:180 -#23 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#24 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#25 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9646e90, f=...) at src/runtime/context.rs:180 -#26 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#27 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9647098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#28 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#29 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#30 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9647220) at src/runtime/blocking/task.rs:42 -#31 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea3140a8) at src/runtime/task/core.rs:328 -#32 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea3140a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#33 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea3140a0, cx=...) at src/runtime/task/core.rs:317 -#34 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#35 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#36 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9647398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#37 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#38 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#39 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#40 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea3140a0, cx=...) at src/runtime/task/harness.rs:473 -#41 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c96475b0) at src/runtime/task/harness.rs:208 -#42 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#43 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#44 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#45 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#46 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#47 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f9247e09150, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 -#48 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#49 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#50 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#51 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#52 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9647a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#53 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#54 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#55 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#56 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#57 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#58 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#59 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#60 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#61 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#62 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1338 (Thread 0x7f91c987c640 (LWP 2741340) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9878750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9878750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9878730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674687776, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bfb20, mutex=0x7f91ea1bfb28, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bfb20, mutex_guard=0x7f91c9847628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bfb20, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bfb10) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bfb10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c98479a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9847e98, core=0x7f956917afe0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9847e98, core=0x7f956917afe0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9847e98, core=0x7f956917afe0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9849610, t=0x7f91c9847e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c98495d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9847e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9848098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9848220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21ff28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21ff28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21ff20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9848398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21ff20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c98485b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=143) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9848a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1337 (Thread 0x7f91c9a7d640 (LWP 2741339) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9a79750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9a79750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9a79730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674687680, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bfac0, mutex=0x7f91ea1bfac8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bfac0, mutex_guard=0x7f91c9a48628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bfac0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bfab0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bfab0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c9a489a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9a48e98, core=0x7f956917af90, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9a48e98, core=0x7f956917af90) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9a48e98, core=0x7f956917af90) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9a4a610, t=0x7f91c9a48e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c9a4a5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9a48e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9a49098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9a49220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fea8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fea8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fea0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9a49398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fea0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c9a495b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=142) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9a49a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1336 (Thread 0x7f91c9c7e640 (LWP 2741338) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9c7a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9c7a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9c7a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674687584, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bfa60, mutex=0x7f91ea1bfa68, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bfa60, mutex_guard=0x7f91c9c49628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bfa60, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bfa50) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bfa50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c9c499a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9c49e98, core=0x7f956917af40, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9c49e98, core=0x7f956917af40) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9c49e98, core=0x7f956917af40) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9c4b610, t=0x7f91c9c49e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c9c4b5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9c49e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9c4a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9c4a220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fe28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fe28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fe20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9c4a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fe20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c9c4a5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=141) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9c4aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1335 (Thread 0x7f91c9e7f640 (LWP 2741337) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91c9e7b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91c9e7b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91c9e7b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674687488, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bfa00, mutex=0x7f91ea1bfa08, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bfa00, mutex_guard=0x7f91c9e4a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bfa00, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf9f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf9f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91c9e4a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91c9e4ae98, core=0x7f956917aef0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91c9e4ae98, core=0x7f956917aef0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91c9e4ae98, core=0x7f956917aef0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91c9e4c610, t=0x7f91c9e4ae90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91c9e4c5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91c9e4ae90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91c9e4b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91c9e4b220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fda8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fda8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fda0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91c9e4b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fda0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91c9e4b5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=140) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91c9e4ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1334 (Thread 0x7f91ca080640 (LWP 2741336) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ca07c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ca07c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ca07c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674687392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf9a0, mutex=0x7f91ea1bf9a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf9a0, mutex_guard=0x7f91ca04b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf9a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf990) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf990, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ca04b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ca04be98, core=0x7f956917aea0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ca04be98, core=0x7f956917aea0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ca04be98, core=0x7f956917aea0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ca04d610, t=0x7f91ca04be90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ca04d5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ca04be90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ca04c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ca04c220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fd28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fd28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fd20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ca04c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fd20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ca04c5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=139) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ca04ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1333 (Thread 0x7f91ca281640 (LWP 2741335) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ca27d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ca27d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ca27d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674687296, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf940, mutex=0x7f91ea1bf948, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf940, mutex_guard=0x7f91ca24c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf940, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf930) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf930, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ca24c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ca24ce98, core=0x7f956917ae50, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ca24ce98, core=0x7f956917ae50) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ca24ce98, core=0x7f956917ae50) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ca24e610, t=0x7f91ca24ce90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ca24e5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ca24ce90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ca24d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ca24d220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fca8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fca8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fca0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ca24d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fca0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ca24d5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=138) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ca24da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1332 (Thread 0x7f91ca482640 (LWP 2741334) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ca47e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ca47e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ca47e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674687200, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf8e0, mutex=0x7f91ea1bf8e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf8e0, mutex_guard=0x7f91ca44d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf8e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf8d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf8d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ca44d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ca44de98, core=0x7f956917ae00, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ca44de98, core=0x7f956917ae00) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ca44de98, core=0x7f956917ae00) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ca44f610, t=0x7f91ca44de90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ca44f5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ca44de90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ca44e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ca44e220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fc28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fc28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fc20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ca44e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fc20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ca44e5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=137) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ca44ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1331 (Thread 0x7f91ca683640 (LWP 2741333) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ca67f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ca67f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ca67f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674687104, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf880, mutex=0x7f91ea1bf888, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf880, mutex_guard=0x7f91ca64e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf880, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf870) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf870, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ca64e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ca64ee98, core=0x7f956917adb0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ca64ee98, core=0x7f956917adb0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ca64ee98, core=0x7f956917adb0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ca650610, t=0x7f91ca64ee90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ca6505d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ca64ee90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ca64f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ca64f220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fba8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fba8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fba0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ca64f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fba0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ca64f5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=136) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ca64fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1330 (Thread 0x7f91ca884640 (LWP 2741332) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ca880750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ca880750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ca880730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674687008, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf820, mutex=0x7f91ea1bf828, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf820, mutex_guard=0x7f91ca84f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf820, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf810) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf810, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ca84f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ca84fe98, core=0x7f956917ad60, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ca84fe98, core=0x7f956917ad60) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ca84fe98, core=0x7f956917ad60) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ca851610, t=0x7f91ca84fe90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ca8515d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ca84fe90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ca850098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ca850220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fb28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fb28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fb20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ca850398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fb20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ca8505b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=135) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ca850a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1329 (Thread 0x7f91caa85640 (LWP 2741331) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91caa81750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91caa81750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91caa81730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674686912, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf7c0, mutex=0x7f91ea1bf7c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf7c0, mutex_guard=0x7f91caa50628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf7c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf7b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf7b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91caa509a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91caa50e98, core=0x7f956917ad10, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91caa50e98, core=0x7f956917ad10) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91caa50e98, core=0x7f956917ad10) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91caa52610, t=0x7f91caa50e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91caa525d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91caa50e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91caa51098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91caa51220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21faa8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21faa8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21faa0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91caa51398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21faa0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91caa515b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=134) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91caa51a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1328 (Thread 0x7f91cac86640 (LWP 2741330) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cac82750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cac82750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cac82730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674686816, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf760, mutex=0x7f91ea1bf768, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf760, mutex_guard=0x7f91cac51628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf760, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf750) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf750, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cac519a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cac51e98, core=0x7f956917acc0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cac51e98, core=0x7f956917acc0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cac51e98, core=0x7f956917acc0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cac53610, t=0x7f91cac51e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cac535d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cac51e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cac52098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cac52220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21fa28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21fa28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21fa20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cac52398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21fa20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cac525b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=133) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cac52a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1327 (Thread 0x7f91cae87640 (LWP 2741329) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cae83750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cae83750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cae83730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674686720, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf700, mutex=0x7f91ea1bf708, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf700, mutex_guard=0x7f91cae52628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf700, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf6f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf6f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cae529a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cae52e98, core=0x7f956917ac70, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cae52e98, core=0x7f956917ac70) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cae52e98, core=0x7f956917ac70) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cae54610, t=0x7f91cae52e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cae545d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cae52e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cae53098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cae53220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f9a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f9a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f9a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cae53398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f9a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cae535b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=132) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cae53a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1326 (Thread 0x7f91cb088640 (LWP 2741328) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cb084750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cb084750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cb084730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674686624, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf6a0, mutex=0x7f91ea1bf6a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf6a0, mutex_guard=0x7f91cb053628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf6a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf690) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf690, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cb0539a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cb053e98, core=0x7f956917ac20, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cb053e98, core=0x7f956917ac20) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cb053e98, core=0x7f956917ac20) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cb055610, t=0x7f91cb053e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cb0555d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cb053e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cb054098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cb054220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f928) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f928, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f920, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cb054398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f920, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cb0545b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=131) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cb054a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1325 (Thread 0x7f91cb289640 (LWP 2741327) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cb285750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cb285750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cb285730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674686528, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf640, mutex=0x7f91ea1bf648, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf640, mutex_guard=0x7f91cb254628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf640, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf630) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf630, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cb2549a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cb254e98, core=0x7f956917abd0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cb254e98, core=0x7f956917abd0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cb254e98, core=0x7f956917abd0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cb256610, t=0x7f91cb254e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cb2565d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cb254e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cb255098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cb255220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f8a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f8a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f8a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cb255398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f8a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cb2555b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=130) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cb255a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1324 (Thread 0x7f91cb48a640 (LWP 2741326) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cb486750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cb486750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cb486730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674686432, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf5e0, mutex=0x7f91ea1bf5e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf5e0, mutex_guard=0x7f91cb455628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf5e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf5d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf5d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cb4559a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cb455e98, core=0x7f956917ab80, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cb455e98, core=0x7f956917ab80) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cb455e98, core=0x7f956917ab80) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cb457610, t=0x7f91cb455e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cb4575d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cb455e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cb456098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cb456220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f828) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f828, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f820, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cb456398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f820, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cb4565b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=129) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cb456a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1323 (Thread 0x7f91cb68b640 (LWP 2741325) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cb687750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cb687750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cb687730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674686336, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf580, mutex=0x7f91ea1bf588, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf580, mutex_guard=0x7f91cb656628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf580, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf570) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf570, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cb6569a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cb656e98, core=0x7f956917ab30, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cb656e98, core=0x7f956917ab30) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cb656e98, core=0x7f956917ab30) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cb658610, t=0x7f91cb656e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cb6585d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cb656e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cb657098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cb657220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f7a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f7a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f7a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cb657398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f7a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cb6575b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=128) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cb657a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1322 (Thread 0x7f91cb88c640 (LWP 2741324) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cb888750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cb888750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cb888730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674686240, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf520, mutex=0x7f91ea1bf528, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf520, mutex_guard=0x7f91cb857628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf520, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf510) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf510, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cb8579a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cb857e98, core=0x7f956917aae0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cb857e98, core=0x7f956917aae0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cb857e98, core=0x7f956917aae0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cb859610, t=0x7f91cb857e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cb8595d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cb857e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cb858098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cb858220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f728) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f728, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f720, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cb858398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f720, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cb8585b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=127) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cb858a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1321 (Thread 0x7f91cba8d640 (LWP 2741323) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cba89750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cba89750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cba89730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674686144, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf4c0, mutex=0x7f91ea1bf4c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf4c0, mutex_guard=0x7f91cba58628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf4c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf4b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf4b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cba589a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cba58e98, core=0x7f956917aa90, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cba58e98, core=0x7f956917aa90) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cba58e98, core=0x7f956917aa90) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cba5a610, t=0x7f91cba58e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cba5a5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cba58e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cba59098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cba59220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f6a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f6a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f6a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cba59398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f6a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cba595b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=126) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cba59a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1320 (Thread 0x7f91cbc8e640 (LWP 2741322) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cbc8a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cbc8a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cbc8a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674686048, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf460, mutex=0x7f91ea1bf468, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf460, mutex_guard=0x7f91cbc59628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf460, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf450) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf450, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cbc599a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cbc59e98, core=0x7f956917aa40, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cbc59e98, core=0x7f956917aa40) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cbc59e98, core=0x7f956917aa40) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cbc5b610, t=0x7f91cbc59e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cbc5b5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cbc59e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cbc5a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cbc5a220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f628) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f628, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f620, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cbc5a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f620, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cbc5a5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=125) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cbc5aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1319 (Thread 0x7f91cbe8f640 (LWP 2741321) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cbe8b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cbe8b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cbe8b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674685856, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf3a0, mutex=0x7f91ea1bf3a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf3a0, mutex_guard=0x7f91cbe5a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf3a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf390) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf390, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cbe5a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cbe5ae98, core=0x7f956917a9a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cbe5ae98, core=0x7f956917a9a0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cbe5ae98, core=0x7f956917a9a0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cbe5c610, t=0x7f91cbe5ae90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cbe5c5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cbe5ae90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cbe5b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cbe5b220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f528) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f528, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f520, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cbe5b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f520, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cbe5b5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=124) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cbe5ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1318 (Thread 0x7f91cc090640 (LWP 2741320) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cc08c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cc08c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cc08c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674685952, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf400, mutex=0x7f91ea1bf408, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf400, mutex_guard=0x7f91cc05b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf400, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf3f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf3f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cc05b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cc05be98, core=0x7f956917a9f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cc05be98, core=0x7f956917a9f0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cc05be98, core=0x7f956917a9f0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cc05d610, t=0x7f91cc05be90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cc05d5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cc05be90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cc05c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cc05c220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea21f5a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea21f5a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea21f5a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cc05c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea21f5a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cc05c5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=123) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cc05ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1317 (Thread 0x7f91cc291640 (LWP 2741319) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cc28d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cc28d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cc28d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674685760, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf340, mutex=0x7f91ea1bf348, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf340, mutex_guard=0x7f91cc25c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf340, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf330) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf330, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cc25c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cc25ce98, core=0x7f956917a950, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cc25ce98, core=0x7f956917a950) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cc25ce98, core=0x7f956917a950) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cc25e610, t=0x7f91cc25ce90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cc25e5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cc25ce90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cc25d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cc25d220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2184a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2184a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2184a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cc25d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2184a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cc25d5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=122) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cc25da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1316 (Thread 0x7f91cc492640 (LWP 2741318) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cc48e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cc48e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cc48e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674685664, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf2e0, mutex=0x7f91ea1bf2e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf2e0, mutex_guard=0x7f91cc45d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf2e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf2d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf2d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cc45d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cc45de98, core=0x7f956917a900, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cc45de98, core=0x7f956917a900) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cc45de98, core=0x7f956917a900) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cc45f610, t=0x7f91cc45de90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cc45f5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cc45de90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cc45e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cc45e220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea218428) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea218428, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea218420, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cc45e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea218420, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cc45e5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=121) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cc45ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1315 (Thread 0x7f91cc693640 (LWP 2741317) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cc68f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cc68f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cc68f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674685568, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf280, mutex=0x7f91ea1bf288, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf280, mutex_guard=0x7f91cc65e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf280, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf270) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf270, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cc65e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cc65ee98, core=0x7f956917a8b0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cc65ee98, core=0x7f956917a8b0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cc65ee98, core=0x7f956917a8b0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cc660610, t=0x7f91cc65ee90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cc6605d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cc65ee90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cc65f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cc65f220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2183a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2183a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2183a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cc65f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2183a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cc65f5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=120) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cc65fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1314 (Thread 0x7f91cc894640 (LWP 2741316) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cc890750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cc890750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cc890730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674685472, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf220, mutex=0x7f91ea1bf228, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf220, mutex_guard=0x7f91cc85f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf220, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf210) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf210, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cc85f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cc85fe98, core=0x7f956917a860, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cc85fe98, core=0x7f956917a860) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cc85fe98, core=0x7f956917a860) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cc861610, t=0x7f91cc85fe90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cc8615d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cc85fe90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cc860098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cc860220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea218328) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea218328, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea218320, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cc860398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea218320, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cc8605b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=119) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cc860a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1313 (Thread 0x7f91cca95640 (LWP 2741315) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cca91750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cca91750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cca91730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674685280, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf160, mutex=0x7f91ea1bf168, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf160, mutex_guard=0x7f91cca60628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf160, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf150) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf150, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cca609a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cca60e98, core=0x7f956917a7c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cca60e98, core=0x7f956917a7c0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cca60e98, core=0x7f956917a7c0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cca62610, t=0x7f91cca60e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cca625d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cca60e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cca61098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cca61220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea218228) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea218228, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea218220, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cca61398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea218220, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cca615b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=118) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cca61a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1312 (Thread 0x7f91ccc96640 (LWP 2741314) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ccc92750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ccc92750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ccc92730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674685376, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf1c0, mutex=0x7f91ea1bf1c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf1c0, mutex_guard=0x7f91ccc61628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf1c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf1b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf1b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ccc619a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ccc61e98, core=0x7f956917a810, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ccc61e98, core=0x7f956917a810) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ccc61e98, core=0x7f956917a810) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ccc63610, t=0x7f91ccc61e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ccc635d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ccc61e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ccc62098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ccc62220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2182a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2182a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2182a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ccc62398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2182a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ccc625b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=117) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ccc62a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1311 (Thread 0x7f91cce97640 (LWP 2741313) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cce93750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cce93750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cce93730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674685184, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf100, mutex=0x7f91ea1bf108, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf100, mutex_guard=0x7f91cce62628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf100, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf0f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf0f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cce629a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cce62e98, core=0x7f956917a770, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cce62e98, core=0x7f956917a770) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cce62e98, core=0x7f956917a770) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cce64610, t=0x7f91cce62e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cce645d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cce62e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cce63098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cce63220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2181a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2181a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2181a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cce63398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2181a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cce635b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=116) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cce63a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1310 (Thread 0x7f91cd098640 (LWP 2741312) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cd094750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cd094750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cd094730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674685088, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf0a0, mutex=0x7f91ea1bf0a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf0a0, mutex_guard=0x7f91cd063628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf0a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf090) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf090, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cd0639a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cd063e98, core=0x7f956917a720, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cd063e98, core=0x7f956917a720) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cd063e98, core=0x7f956917a720) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cd065610, t=0x7f91cd063e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cd0655d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cd063e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cd064098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cd064220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea218128) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea218128, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea218120, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cd064398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea218120, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cd0645b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=115) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cd064a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1309 (Thread 0x7f91cd299640 (LWP 2741311) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cd295750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cd295750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cd295730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684992, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bf040, mutex=0x7f91ea1bf048, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bf040, mutex_guard=0x7f91cd264628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bf040, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bf030) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bf030, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cd2649a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cd264e98, core=0x7f956917a6d0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cd264e98, core=0x7f956917a6d0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cd264e98, core=0x7f956917a6d0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cd266610, t=0x7f91cd264e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cd2665d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cd264e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cd265098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cd265220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2180a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2180a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2180a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cd265398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2180a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cd2655b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=114) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cd265a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1308 (Thread 0x7f91cd49a640 (LWP 2741310) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cd496750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cd496750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cd496730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684896, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1befe0, mutex=0x7f91ea1befe8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1befe0, mutex_guard=0x7f91cd465628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1befe0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1befd0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1befd0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cd4659a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cd465e98, core=0x7f956917a680, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cd465e98, core=0x7f956917a680) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cd465e98, core=0x7f956917a680) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cd467610, t=0x7f91cd465e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cd4675d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cd465e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cd466098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cd466220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea218028) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea218028, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea218020, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cd466398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea218020, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cd4665b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=113) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cd466a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1307 (Thread 0x7f91cd69b640 (LWP 2741309) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cd697750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cd697750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cd697730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684800, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bef80, mutex=0x7f91ea1bef88, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bef80, mutex_guard=0x7f91cd666628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bef80, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bef70) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bef70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cd6669a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cd666e98, core=0x7f956917a630, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cd666e98, core=0x7f956917a630) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cd666e98, core=0x7f956917a630) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cd668610, t=0x7f91cd666e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cd6685d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cd666e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cd667098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cd667220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211fa8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211fa8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211fa0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cd667398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211fa0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cd6675b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=112) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cd667a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1306 (Thread 0x7f91cd89c640 (LWP 2741308) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cd898750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cd898750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cd898730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684704, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bef20, mutex=0x7f91ea1bef28, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bef20, mutex_guard=0x7f91cd867628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bef20, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bef10) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bef10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cd8679a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cd867e98, core=0x7f956917a5e0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cd867e98, core=0x7f956917a5e0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cd867e98, core=0x7f956917a5e0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cd869610, t=0x7f91cd867e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cd8695d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cd867e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cd868098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cd868220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211f28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211f28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211f20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cd868398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211f20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cd8685b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=111) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cd868a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1305 (Thread 0x7f91cda9d640 (LWP 2741307) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cda99750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cda99750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cda99730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684608, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1beec0, mutex=0x7f91ea1beec8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1beec0, mutex_guard=0x7f91cda68628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1beec0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1beeb0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1beeb0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cda689a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cda68e98, core=0x7f956917a590, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cda68e98, core=0x7f956917a590) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cda68e98, core=0x7f956917a590) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cda6a610, t=0x7f91cda68e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cda6a5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cda68e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cda69098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cda69220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211ea8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211ea8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211ea0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cda69398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211ea0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cda695b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=110) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cda69a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1304 (Thread 0x7f91cdc9e640 (LWP 2741306) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cdc9a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cdc9a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cdc9a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684512, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bee60, mutex=0x7f91ea1bee68, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bee60, mutex_guard=0x7f91cdc69628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bee60, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bee50) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bee50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cdc699a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cdc69e98, core=0x7f956917a540, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cdc69e98, core=0x7f956917a540) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cdc69e98, core=0x7f956917a540) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cdc6b610, t=0x7f91cdc69e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cdc6b5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cdc69e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cdc6a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cdc6a220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211e28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211e28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211e20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cdc6a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211e20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cdc6a5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=109) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cdc6aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1303 (Thread 0x7f91cde9f640 (LWP 2741305) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cde9b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cde9b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cde9b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684416, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bee00, mutex=0x7f91ea1bee08, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bee00, mutex_guard=0x7f91cde6a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bee00, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bedf0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bedf0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cde6a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cde6ae98, core=0x7f956917a4f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cde6ae98, core=0x7f956917a4f0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cde6ae98, core=0x7f956917a4f0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cde6c610, t=0x7f91cde6ae90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cde6c5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cde6ae90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cde6b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cde6b220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211da8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211da8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211da0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cde6b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211da0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cde6b5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=108) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cde6ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1302 (Thread 0x7f91ce0a0640 (LWP 2741304) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ce09c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ce09c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ce09c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684224, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bed40, mutex=0x7f91ea1bed48, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bed40, mutex_guard=0x7f91ce06b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bed40, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bed30) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bed30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ce06b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ce06be98, core=0x7f956917a450, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ce06be98, core=0x7f956917a450) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ce06be98, core=0x7f956917a450) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ce06d610, t=0x7f91ce06be90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ce06d5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ce06be90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ce06c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ce06c220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211ca8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211ca8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211ca0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ce06c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211ca0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ce06c5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=107) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ce06ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1301 (Thread 0x7f91ce2a1640 (LWP 2741303) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ce29d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ce29d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ce29d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684320, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1beda0, mutex=0x7f91ea1beda8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1beda0, mutex_guard=0x7f91ce26c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1beda0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bed90) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bed90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ce26c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ce26ce98, core=0x7f956917a4a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ce26ce98, core=0x7f956917a4a0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ce26ce98, core=0x7f956917a4a0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ce26e610, t=0x7f91ce26ce90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ce26e5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ce26ce90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ce26d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ce26d220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211d28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211d28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211d20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ce26d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211d20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ce26d5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=106) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ce26da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1300 (Thread 0x7f91ce4a2640 (LWP 2741302) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ce49e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ce49e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ce49e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684128, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bece0, mutex=0x7f91ea1bece8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bece0, mutex_guard=0x7f91ce46d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bece0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1becd0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1becd0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ce46d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ce46de98, core=0x7f956917a400, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ce46de98, core=0x7f956917a400) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ce46de98, core=0x7f956917a400) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ce46f610, t=0x7f91ce46de90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ce46f5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ce46de90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ce46e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ce46e220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211c28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211c28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211c20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ce46e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211c20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ce46e5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=105) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ce46ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1299 (Thread 0x7f91ce6a3640 (LWP 2741301) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ce69f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ce69f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ce69f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674684032, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bec80, mutex=0x7f91ea1bec88, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bec80, mutex_guard=0x7f91ce66e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bec80, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bec70) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bec70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ce66e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ce66ee98, core=0x7f956917a3b0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ce66ee98, core=0x7f956917a3b0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ce66ee98, core=0x7f956917a3b0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ce670610, t=0x7f91ce66ee90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ce6705d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ce66ee90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ce66f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ce66f220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211ba8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211ba8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211ba0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ce66f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211ba0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ce66f5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=104) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ce66fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1298 (Thread 0x7f91ce8a4640 (LWP 2741300) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ce8a0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ce8a0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ce8a0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674683936, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bec20, mutex=0x7f91ea1bec28, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bec20, mutex_guard=0x7f91ce86f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bec20, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bec10) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bec10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ce86f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ce86fe98, core=0x7f956917a360, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ce86fe98, core=0x7f956917a360) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ce86fe98, core=0x7f956917a360) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ce871610, t=0x7f91ce86fe90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ce8715d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ce86fe90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ce870098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ce870220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211b28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211b28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211b20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ce870398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211b20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ce8705b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=103) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ce870a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1297 (Thread 0x7f91ced25640 (LWP 2741299) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ced21750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ced21750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91ced21730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674683840, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bebc0, mutex=0x7f91ea1bebc8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bebc0, mutex_guard=0x7f91cecf0628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bebc0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bebb0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bebb0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cecf09a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cecf0e98, core=0x7f956917a310, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cecf0e98, core=0x7f956917a310) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cecf0e98, core=0x7f956917a310) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cecf2610, t=0x7f91cecf0e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cecf25d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cecf0e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cecf1098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cecf1220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211aa8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211aa8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211aa0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cecf1398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211aa0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cecf15b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=102) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cecf1a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1296 (Thread 0x7f91cef26640 (LWP 2741298) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cef22750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cef22750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cef22730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674683744, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1beb60, mutex=0x7f91ea1beb68, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1beb60, mutex_guard=0x7f91ceef1628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1beb60, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1beb50) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1beb50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91ceef19a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91ceef1e98, core=0x7f956917a2c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91ceef1e98, core=0x7f956917a2c0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91ceef1e98, core=0x7f956917a2c0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91ceef3610, t=0x7f91ceef1e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91ceef35d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91ceef1e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91ceef2098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91ceef2220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211a28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211a28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211a20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91ceef2398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211a20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ceef25b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=101) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91ceef2a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1295 (Thread 0x7f91cf127640 (LWP 2741297) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cf123750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cf123750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cf123730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674683648, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1beb00, mutex=0x7f91ea1beb08, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1beb00, mutex_guard=0x7f91cf0f2628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1beb00, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1beaf0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1beaf0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cf0f29a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cf0f2e98, core=0x7f956917a270, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cf0f2e98, core=0x7f956917a270) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cf0f2e98, core=0x7f956917a270) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cf0f4610, t=0x7f91cf0f2e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cf0f45d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cf0f2e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cf0f3098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cf0f3220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2119a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2119a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2119a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cf0f3398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2119a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cf0f35b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=100) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cf0f3a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1294 (Thread 0x7f91cf328640 (LWP 2741296) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cf324750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cf324750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cf324730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674683552, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1beaa0, mutex=0x7f91ea1beaa8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1beaa0, mutex_guard=0x7f91cf2f3628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1beaa0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bea90) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bea90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cf2f39a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cf2f3e98, core=0x7f956917a220, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cf2f3e98, core=0x7f956917a220) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cf2f3e98, core=0x7f956917a220) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cf2f5610, t=0x7f91cf2f3e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cf2f55d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cf2f3e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cf2f4098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cf2f4220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211928) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211928, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211920, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cf2f4398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211920, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cf2f45b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=99) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cf2f4a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1293 (Thread 0x7f91cf529640 (LWP 2741295) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91cf525750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91cf525750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91cf525730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674683360, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be9e0, mutex=0x7f91ea1be9e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be9e0, mutex_guard=0x7f91cf4f4628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be9e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be9d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be9d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91cf4f49a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91cf4f4e98, core=0x7f956917a180, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91cf4f4e98, core=0x7f956917a180) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91cf4f4e98, core=0x7f956917a180) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91cf4f6610, t=0x7f91cf4f4e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91cf4f65d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91cf4f4e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91cf4f5098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91cf4f5220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211828) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211828, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211820, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91cf4f5398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211820, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91cf4f55b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=98) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91cf4f5a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1292 (Thread 0x7f91d0b2a640 (LWP 2741294) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d0b26750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d0b26750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d0b26730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674683264, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be980, mutex=0x7f91ea1be988, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be980, mutex_guard=0x7f91d0af5628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be980, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be970) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be970, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d0af59a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d0af5e98, core=0x7f956917a130, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d0af5e98, core=0x7f956917a130) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d0af5e98, core=0x7f956917a130) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d0af7610, t=0x7f91d0af5e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d0af75d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d0af5e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d0af6098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d0af6220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2117a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2117a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2117a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d0af6398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2117a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d0af65b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=97) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d0af6a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1291 (Thread 0x7f91d0d2b640 (LWP 2741293) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d0d27750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d0d27750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d0d27730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674683168, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be920, mutex=0x7f91ea1be928, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be920, mutex_guard=0x7f91d0cf6628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be920, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be910) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be910, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d0cf69a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d0cf6e98, core=0x7f956917a0e0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d0cf6e98, core=0x7f956917a0e0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d0cf6e98, core=0x7f956917a0e0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d0cf8610, t=0x7f91d0cf6e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d0cf85d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d0cf6e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d0cf7098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d0cf7220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211728) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211728, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211720, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d0cf7398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211720, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d0cf75b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=96) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d0cf7a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1290 (Thread 0x7f91d0f2c640 (LWP 2741292) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d0f28750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d0f28750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d0f28730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674683072, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be8c0, mutex=0x7f91ea1be8c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be8c0, mutex_guard=0x7f91d0ef7628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be8c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be8b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be8b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d0ef79a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d0ef7e98, core=0x7f956917a090, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d0ef7e98, core=0x7f956917a090) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d0ef7e98, core=0x7f956917a090) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d0ef9610, t=0x7f91d0ef7e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d0ef95d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d0ef7e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d0ef8098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d0ef8220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2116a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2116a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2116a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d0ef8398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2116a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d0ef85b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=95) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d0ef8a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1289 (Thread 0x7f91d112d640 (LWP 2741291) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d1129750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d1129750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d1129730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682976, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be860, mutex=0x7f91ea1be868, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be860, mutex_guard=0x7f91d10f8628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be860, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be850) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be850, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d10f89a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d10f8e98, core=0x7f956917a040, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d10f8e98, core=0x7f956917a040) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d10f8e98, core=0x7f956917a040) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d10fa610, t=0x7f91d10f8e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d10fa5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d10f8e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d10f9098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d10f9220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211628) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211628, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211620, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d10f9398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211620, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d10f95b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=94) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d10f9a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1288 (Thread 0x7f91d132e640 (LWP 2741290) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d132a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d132a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d132a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674683456, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1bea40, mutex=0x7f91ea1bea48, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1bea40, mutex_guard=0x7f91d12f9628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1bea40, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1bea30) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1bea30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d12f99a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d12f9e98, core=0x7f956917a1d0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d12f9e98, core=0x7f956917a1d0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d12f9e98, core=0x7f956917a1d0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d12fb610, t=0x7f91d12f9e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d12fb5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d12f9e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d12fa098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d12fa220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2118a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2118a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2118a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d12fa398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2118a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d12fa5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=93) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d12faa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1287 (Thread 0x7f91d152f640 (LWP 2741289) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d152b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d152b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d152b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682880, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be800, mutex=0x7f91ea1be808, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be800, mutex_guard=0x7f91d14fa628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be800, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be7f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be7f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d14fa9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d14fae98, core=0x7f9569179ff0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d14fae98, core=0x7f9569179ff0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d14fae98, core=0x7f9569179ff0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d14fc610, t=0x7f91d14fae90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d14fc5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d14fae90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d14fb098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d14fb220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2115a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2115a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2115a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d14fb398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2115a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d14fb5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=92) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d14fba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1286 (Thread 0x7f91d1730640 (LWP 2741288) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d172c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d172c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d172c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682784, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be7a0, mutex=0x7f91ea1be7a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be7a0, mutex_guard=0x7f91d16fb628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be7a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be790) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be790, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d16fb9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d16fbe98, core=0x7f9569179fa0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d16fbe98, core=0x7f9569179fa0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d16fbe98, core=0x7f9569179fa0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d16fd610, t=0x7f91d16fbe90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d16fd5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d16fbe90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d16fc098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d16fc220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211528) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211528, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211520, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d16fc398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211520, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d16fc5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=91) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d16fca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1285 (Thread 0x7f91d1931640 (LWP 2741287) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d192d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d192d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d192d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682688, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be740, mutex=0x7f91ea1be748, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be740, mutex_guard=0x7f91d18fc628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be740, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be730) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be730, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d18fc9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d18fce98, core=0x7f9569179f50, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d18fce98, core=0x7f9569179f50) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d18fce98, core=0x7f9569179f50) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d18fe610, t=0x7f91d18fce90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d18fe5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d18fce90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d18fd098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d18fd220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2114a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2114a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2114a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d18fd398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2114a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d18fd5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=90) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d18fda20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1284 (Thread 0x7f91d1b32640 (LWP 2741286) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d1b2e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d1b2e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d1b2e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be6e0, mutex=0x7f91ea1be6e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be6e0, mutex_guard=0x7f91d1afd628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be6e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be6d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be6d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d1afd9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d1afde98, core=0x7f9569179f00, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d1afde98, core=0x7f9569179f00) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d1afde98, core=0x7f9569179f00) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d1aff610, t=0x7f91d1afde90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d1aff5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d1afde90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d1afe098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d1afe220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211428) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211428, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211420, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d1afe398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211420, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d1afe5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=89) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d1afea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1283 (Thread 0x7f91d1d33640 (LWP 2741285) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d1d2f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d1d2f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d1d2f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682496, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be680, mutex=0x7f91ea1be688, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be680, mutex_guard=0x7f91d1cfe628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be680, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be670) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be670, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d1cfe9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d1cfee98, core=0x7f9569179eb0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d1cfee98, core=0x7f9569179eb0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d1cfee98, core=0x7f9569179eb0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d1d00610, t=0x7f91d1cfee90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d1d005d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d1cfee90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d1cff098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d1cff220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2113a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2113a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2113a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d1cff398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2113a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d1cff5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=88) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d1cffa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1282 (Thread 0x7f91d1f34640 (LWP 2741284) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d1f30750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d1f30750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d1f30730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682400, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be620, mutex=0x7f91ea1be628, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be620, mutex_guard=0x7f91d1eff628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be620, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be610) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be610, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d1eff9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d1effe98, core=0x7f9569179e60, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d1effe98, core=0x7f9569179e60) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d1effe98, core=0x7f9569179e60) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d1f01610, t=0x7f91d1effe90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d1f015d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d1effe90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d1f00098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d1f00220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211328) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211328, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211320, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d1f00398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211320, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d1f005b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=87) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d1f00a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1281 (Thread 0x7f91d2135640 (LWP 2741283) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2131750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2131750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2131730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682304, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be5c0, mutex=0x7f91ea1be5c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be5c0, mutex_guard=0x7f91d2100628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be5c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be5b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be5b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d21009a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2100e98, core=0x7f9569179e10, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2100e98, core=0x7f9569179e10) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2100e98, core=0x7f9569179e10) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2102610, t=0x7f91d2100e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d21025d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2100e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2101098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2101220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2112a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2112a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2112a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2101398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2112a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d21015b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=86) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2101a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1280 (Thread 0x7f91d2336640 (LWP 2741282) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2332750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2332750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2332730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682208, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be560, mutex=0x7f91ea1be568, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be560, mutex_guard=0x7f91d2301628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be560, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be550) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be550, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d23019a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2301e98, core=0x7f9569179dc0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2301e98, core=0x7f9569179dc0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2301e98, core=0x7f9569179dc0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2303610, t=0x7f91d2301e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d23035d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2301e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2302098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2302220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211228) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211228, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211220, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2302398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211220, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d23025b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=85) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2302a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1279 (Thread 0x7f91d2537640 (LWP 2741281) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2533750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2533750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2533730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682112, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be500, mutex=0x7f91ea1be508, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be500, mutex_guard=0x7f91d2502628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be500, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be4f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be4f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d25029a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2502e98, core=0x7f9569179d70, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2502e98, core=0x7f9569179d70) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2502e98, core=0x7f9569179d70) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2504610, t=0x7f91d2502e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d25045d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2502e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2503098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2503220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2111a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2111a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2111a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2503398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2111a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d25035b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=84) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2503a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1278 (Thread 0x7f91d2738640 (LWP 2741280) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2734750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2734750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2734730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674682016, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be4a0, mutex=0x7f91ea1be4a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be4a0, mutex_guard=0x7f91d2703628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be4a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be490) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be490, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d27039a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2703e98, core=0x7f9569179d20, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2703e98, core=0x7f9569179d20) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2703e98, core=0x7f9569179d20) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2705610, t=0x7f91d2703e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d27055d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2703e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2704098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2704220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211128) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211128, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211120, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2704398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211120, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d27045b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=83) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2704a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1277 (Thread 0x7f91d2939640 (LWP 2741279) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2935750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2935750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2935730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674681920, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be440, mutex=0x7f91ea1be448, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be440, mutex_guard=0x7f91d2904628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be440, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be430) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be430, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d29049a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2904e98, core=0x7f9569179cd0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2904e98, core=0x7f9569179cd0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2904e98, core=0x7f9569179cd0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2906610, t=0x7f91d2904e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d29065d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2904e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2905098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2905220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea2110a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea2110a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea2110a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2905398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea2110a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d29055b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=82) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2905a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1276 (Thread 0x7f91d2b3a640 (LWP 2741278) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2b36750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2b36750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2b36730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674681728, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be380, mutex=0x7f91ea1be388, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be380, mutex_guard=0x7f91d2b05628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be380, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be370) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be370, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d2b059a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2b05e98, core=0x7f9569179c30, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2b05e98, core=0x7f9569179c30) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2b05e98, core=0x7f9569179c30) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2b07610, t=0x7f91d2b05e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d2b075d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2b05e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2b06098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2b06220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bfa8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bfa8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bfa0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2b06398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bfa0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d2b065b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=81) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2b06a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1275 (Thread 0x7f91d2d3b640 (LWP 2741277) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2d37750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2d37750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2d37730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674681824, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be3e0, mutex=0x7f91ea1be3e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be3e0, mutex_guard=0x7f91d2d06628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be3e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be3d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be3d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d2d069a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2d06e98, core=0x7f9569179c80, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2d06e98, core=0x7f9569179c80) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2d06e98, core=0x7f9569179c80) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2d08610, t=0x7f91d2d06e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d2d085d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2d06e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2d07098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2d07220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea211028) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea211028, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea211020, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2d07398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea211020, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d2d075b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=80) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2d07a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1274 (Thread 0x7f91d2f3c640 (LWP 2741276) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d2f38750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d2f38750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d2f38730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674681632, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be320, mutex=0x7f91ea1be328, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be320, mutex_guard=0x7f91d2f07628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be320, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be310) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be310, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d2f079a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d2f07e98, core=0x7f9569179be0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d2f07e98, core=0x7f9569179be0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d2f07e98, core=0x7f9569179be0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d2f09610, t=0x7f91d2f07e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d2f095d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d2f07e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d2f08098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d2f08220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bf28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bf28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bf20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d2f08398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bf20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d2f085b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=79) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d2f08a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1273 (Thread 0x7f91d313d640 (LWP 2741275) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d3139750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d3139750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d3139730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674681536, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be2c0, mutex=0x7f91ea1be2c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be2c0, mutex_guard=0x7f91d3108628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be2c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be2b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be2b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d31089a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d3108e98, core=0x7f9569179b90, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d3108e98, core=0x7f9569179b90) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d3108e98, core=0x7f9569179b90) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d310a610, t=0x7f91d3108e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d310a5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d3108e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d3109098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d3109220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bea8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bea8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bea0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d3109398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bea0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d31095b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=78) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d3109a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1272 (Thread 0x7f91d333e640 (LWP 2741274) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d333a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d333a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d333a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674681440, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be260, mutex=0x7f91ea1be268, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be260, mutex_guard=0x7f91d3309628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be260, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be250) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be250, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d33099a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d3309e98, core=0x7f9569179b40, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d3309e98, core=0x7f9569179b40) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d3309e98, core=0x7f9569179b40) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d330b610, t=0x7f91d3309e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d330b5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d3309e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d330a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d330a220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20be28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20be28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20be20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d330a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20be20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d330a5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=77) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d330aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1271 (Thread 0x7f91d353f640 (LWP 2741273) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d353b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d353b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d353b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674681344, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be200, mutex=0x7f91ea1be208, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be200, mutex_guard=0x7f91d350a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be200, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be1f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be1f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d350a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d350ae98, core=0x7f9569179af0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d350ae98, core=0x7f9569179af0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d350ae98, core=0x7f9569179af0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d350c610, t=0x7f91d350ae90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d350c5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d350ae90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d350b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d350b220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bda8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bda8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bda0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d350b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bda0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d350b5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=76) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d350ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1270 (Thread 0x7f91d3740640 (LWP 2741272) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d373c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d373c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d373c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674681248, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be1a0, mutex=0x7f91ea1be1a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be1a0, mutex_guard=0x7f91d370b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be1a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be190) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be190, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d370b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d370be98, core=0x7f9569179aa0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d370be98, core=0x7f9569179aa0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d370be98, core=0x7f9569179aa0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d370d610, t=0x7f91d370be90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d370d5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d370be90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d370c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d370c220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bd28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bd28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bd20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d370c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bd20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d370c5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=75) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d370ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1269 (Thread 0x7f91d3941640 (LWP 2741271) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d393d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d393d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d393d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674681152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be140, mutex=0x7f91ea1be148, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be140, mutex_guard=0x7f91d390c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be140, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be130) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be130, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d390c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d390ce98, core=0x7f9569179a50, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d390ce98, core=0x7f9569179a50) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d390ce98, core=0x7f9569179a50) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d390e610, t=0x7f91d390ce90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d390e5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d390ce90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d390d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d390d220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bca8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bca8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bca0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d390d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bca0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d390d5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=74) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d390da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1268 (Thread 0x7f91d3b42640 (LWP 2741270) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d3b3e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d3b3e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d3b3e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674680960, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be080, mutex=0x7f91ea1be088, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be080, mutex_guard=0x7f91d3b0d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be080, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be070) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be070, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d3b0d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d3b0de98, core=0x7f95691799b0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d3b0de98, core=0x7f95691799b0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d3b0de98, core=0x7f95691799b0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d3b0f610, t=0x7f91d3b0de90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d3b0f5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d3b0de90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d3b0e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d3b0e220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bba8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bba8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bba0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d3b0e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bba0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d3b0e5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=73) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d3b0ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1267 (Thread 0x7f91d3d43640 (LWP 2741269) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d3d3f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d3d3f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d3d3f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674681056, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be0e0, mutex=0x7f91ea1be0e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be0e0, mutex_guard=0x7f91d3d0e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be0e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be0d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be0d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d3d0e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d3d0ee98, core=0x7f9569179a00, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d3d0ee98, core=0x7f9569179a00) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d3d0ee98, core=0x7f9569179a00) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d3d10610, t=0x7f91d3d0ee90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d3d105d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d3d0ee90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d3d0f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d3d0f220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bc28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bc28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bc20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d3d0f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bc20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d3d0f5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=72) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d3d0fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1266 (Thread 0x7f91d3f44640 (LWP 2741268) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d3f40750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d3f40750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d3f40730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264674680864, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea1be020, mutex=0x7f91ea1be028, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea1be020, mutex_guard=0x7f91d3f0f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea1be020, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea1be010) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea1be010, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d3f0f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d3f0fe98, core=0x7f9569179960, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d3f0fe98, core=0x7f9569179960) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d3f0fe98, core=0x7f9569179960) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d3f11610, t=0x7f91d3f0fe90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d3f115d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d3f0fe90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d3f10098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d3f10220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20bb28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20bb28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20bb20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d3f10398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20bb20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d3f105b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=71) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d3f10a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1265 (Thread 0x7f91d4145640 (LWP 2741267) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4141750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4141750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4141730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238976, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dfc0, mutex=0x7f91ea05dfc8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dfc0, mutex_guard=0x7f91d4110628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dfc0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dfb0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dfb0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d41109a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4110e98, core=0x7f9569179910, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4110e98, core=0x7f9569179910) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4110e98, core=0x7f9569179910) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4112610, t=0x7f91d4110e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d41125d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4110e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4111098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4111220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20baa8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20baa8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20baa0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4111398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20baa0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d41115b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=70) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4111a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1264 (Thread 0x7f91d4346640 (LWP 2741266) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4342750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4342750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4342730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238784, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05df00, mutex=0x7f91ea05df08, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05df00, mutex_guard=0x7f91d4311628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05df00, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05def0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05def0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d43119a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4311e98, core=0x7f9569179870, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4311e98, core=0x7f9569179870) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4311e98, core=0x7f9569179870) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4313610, t=0x7f91d4311e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d43135d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4311e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4312098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4312220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b9a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b9a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b9a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4312398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b9a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d43125b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=69) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4312a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1263 (Thread 0x7f91d4547640 (LWP 2741265) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4543750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4543750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4543730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238880, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05df60, mutex=0x7f91ea05df68, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05df60, mutex_guard=0x7f91d4512628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05df60, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05df50) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05df50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d45129a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4512e98, core=0x7f95691798c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4512e98, core=0x7f95691798c0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4512e98, core=0x7f95691798c0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4514610, t=0x7f91d4512e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d45145d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4512e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4513098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4513220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20ba28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20ba28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20ba20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4513398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20ba20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d45135b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=68) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4513a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1262 (Thread 0x7f91d4748640 (LWP 2741264) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4744750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4744750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4744730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238688, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dea0, mutex=0x7f91ea05dea8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dea0, mutex_guard=0x7f91d4713628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dea0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05de90) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05de90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d47139a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4713e98, core=0x7f9569179820, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4713e98, core=0x7f9569179820) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4713e98, core=0x7f9569179820) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4715610, t=0x7f91d4713e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d47155d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4713e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4714098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4714220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b928) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b928, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b920, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4714398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b920, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d47145b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=67) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4714a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1261 (Thread 0x7f91d4949640 (LWP 2741263) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4945750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4945750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4945730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05de40, mutex=0x7f91ea05de48, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05de40, mutex_guard=0x7f91d4914628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05de40, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05de30) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05de30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d49149a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4914e98, core=0x7f95691797d0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4914e98, core=0x7f95691797d0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4914e98, core=0x7f95691797d0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4916610, t=0x7f91d4914e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d49165d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4914e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4915098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4915220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b8a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b8a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b8a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4915398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b8a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d49155b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=66) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4915a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1260 (Thread 0x7f91d4b4a640 (LWP 2741262) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4b46750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4b46750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4b46730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238496, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dde0, mutex=0x7f91ea05dde8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dde0, mutex_guard=0x7f91d4b15628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dde0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ddd0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ddd0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d4b159a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4b15e98, core=0x7f9569179780, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4b15e98, core=0x7f9569179780) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4b15e98, core=0x7f9569179780) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4b17610, t=0x7f91d4b15e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d4b175d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4b15e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4b16098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4b16220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b828) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b828, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b820, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4b16398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b820, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d4b165b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=65) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4b16a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1259 (Thread 0x7f91d4d4b640 (LWP 2741261) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4d47750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4d47750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4d47730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238400, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dd80, mutex=0x7f91ea05dd88, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dd80, mutex_guard=0x7f91d4d16628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dd80, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dd70) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dd70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d4d169a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4d16e98, core=0x7f9569179730, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4d16e98, core=0x7f9569179730) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4d16e98, core=0x7f9569179730) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4d18610, t=0x7f91d4d16e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d4d185d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4d16e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4d17098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4d17220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b7a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b7a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b7a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4d17398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b7a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d4d175b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=64) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4d17a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1258 (Thread 0x7f91d4f4c640 (LWP 2741260) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d4f48750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d4f48750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d4f48730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238304, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dd20, mutex=0x7f91ea05dd28, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dd20, mutex_guard=0x7f91d4f17628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dd20, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dd10) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dd10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d4f179a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d4f17e98, core=0x7f95691795f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d4f17e98, core=0x7f95691795f0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d4f17e98, core=0x7f95691795f0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d4f19610, t=0x7f91d4f17e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d4f195d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d4f17e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d4f18098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d4f18220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b728) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b728, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b720, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d4f18398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b720, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d4f185b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=63) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d4f18a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1257 (Thread 0x7f91d514d640 (LWP 2741259) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d5149750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d5149750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d5149730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238208, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dcc0, mutex=0x7f91ea05dcc8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dcc0, mutex_guard=0x7f91d5118628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dcc0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dcb0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dcb0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d51189a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d5118e98, core=0x7f95691795a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d5118e98, core=0x7f95691795a0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d5118e98, core=0x7f95691795a0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d511a610, t=0x7f91d5118e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d511a5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d5118e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d5119098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d5119220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b6a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b6a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b6a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d5119398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b6a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d51195b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=62) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d5119a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1256 (Thread 0x7f91d534e640 (LWP 2741258) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d534a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d534a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d534a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238112, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dc60, mutex=0x7f91ea05dc68, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dc60, mutex_guard=0x7f91d5319628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dc60, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dc50) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dc50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d53199a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d5319e98, core=0x7f9569179550, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d5319e98, core=0x7f9569179550) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d5319e98, core=0x7f9569179550) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d531b610, t=0x7f91d5319e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d531b5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d5319e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d531a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d531a220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b628) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b628, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b620, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d531a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b620, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d531a5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=61) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d531aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1255 (Thread 0x7f91d554f640 (LWP 2741257) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d554b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d554b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d554b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673238016, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dc00, mutex=0x7f91ea05dc08, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dc00, mutex_guard=0x7f91d551a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dc00, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dbf0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dbf0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d551a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d551ae98, core=0x7f9569179500, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d551ae98, core=0x7f9569179500) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d551ae98, core=0x7f9569179500) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d551c610, t=0x7f91d551ae90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d551c5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d551ae90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d551b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d551b220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b5a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b5a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b5a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d551b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b5a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d551b5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=60) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d551ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1254 (Thread 0x7f91d5750640 (LWP 2741256) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d574c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d574c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d574c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673237920, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dba0, mutex=0x7f91ea05dba8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dba0, mutex_guard=0x7f91d571b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dba0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05db90) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05db90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d571b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d571be98, core=0x7f95691794b0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d571be98, core=0x7f95691794b0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d571be98, core=0x7f95691794b0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d571d610, t=0x7f91d571be90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d571d5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d571be90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d571c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d571c220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b528) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b528, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b520, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d571c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b520, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d571c5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=59) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d571ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1253 (Thread 0x7f91d5951640 (LWP 2741255) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d594d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d594d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d594d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673237824, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05db40, mutex=0x7f91ea05db48, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05db40, mutex_guard=0x7f91d591c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05db40, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05db30) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05db30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d591c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d591ce98, core=0x7f9569179460, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d591ce98, core=0x7f9569179460) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d591ce98, core=0x7f9569179460) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d591e610, t=0x7f91d591ce90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d591e5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d591ce90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d591d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d591d220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b4a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b4a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b4a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d591d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b4a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d591d5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=58) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d591da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1252 (Thread 0x7f91d5b52640 (LWP 2741254) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d5b4e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d5b4e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d5b4e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673237728, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05dae0, mutex=0x7f91ea05dae8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05dae0, mutex_guard=0x7f91d5b1d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05dae0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05dad0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05dad0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d5b1d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d5b1de98, core=0x7f9569179410, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d5b1de98, core=0x7f9569179410) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d5b1de98, core=0x7f9569179410) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d5b1f610, t=0x7f91d5b1de90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d5b1f5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d5b1de90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d5b1e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d5b1e220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b428) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b428, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b420, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d5b1e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b420, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d5b1e5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=57) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d5b1ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1251 (Thread 0x7f91d5d53640 (LWP 2741253) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d5d4f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d5d4f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d5d4f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673237632, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05da80, mutex=0x7f91ea05da88, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05da80, mutex_guard=0x7f91d5d1e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05da80, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05da70) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05da70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d5d1e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d5d1ee98, core=0x7f95691793c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d5d1ee98, core=0x7f95691793c0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d5d1ee98, core=0x7f95691793c0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d5d20610, t=0x7f91d5d1ee90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d5d205d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d5d1ee90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d5d1f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d5d1f220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b3a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b3a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b3a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d5d1f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b3a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d5d1f5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=56) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d5d1fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1250 (Thread 0x7f91d5f54640 (LWP 2741252) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d5f50750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d5f50750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d5f50730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673237536, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05da20, mutex=0x7f91ea05da28, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05da20, mutex_guard=0x7f91d5f1f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05da20, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05da10) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05da10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d5f1f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d5f1fe98, core=0x7f9569179370, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d5f1fe98, core=0x7f9569179370) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d5f1fe98, core=0x7f9569179370) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d5f21610, t=0x7f91d5f1fe90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d5f215d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d5f1fe90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d5f20098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d5f20220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b328) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b328, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b320, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d5f20398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b320, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d5f205b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=55) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d5f20a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1249 (Thread 0x7f91d6155640 (LWP 2741251) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6151750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6151750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6151730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673237440, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d9c0, mutex=0x7f91ea05d9c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d9c0, mutex_guard=0x7f91d6120628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d9c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d9b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d9b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d61209a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6120e98, core=0x7f9569179320, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6120e98, core=0x7f9569179320) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6120e98, core=0x7f9569179320) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6122610, t=0x7f91d6120e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d61225d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6120e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6121098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6121220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b2a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b2a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b2a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6121398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b2a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d61215b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=54) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6121a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1248 (Thread 0x7f91d6356640 (LWP 2741250) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6352750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6352750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6352730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673237248, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d900, mutex=0x7f91ea05d908, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d900, mutex_guard=0x7f91d6321628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d900, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d8f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d8f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d63219a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6321e98, core=0x7f9569179280, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6321e98, core=0x7f9569179280) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6321e98, core=0x7f9569179280) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6323610, t=0x7f91d6321e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d63235d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6321e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6322098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6322220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b1a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b1a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b1a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6322398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b1a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d63225b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=53) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6322a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1247 (Thread 0x7f91d6557640 (LWP 2741249) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6553750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6553750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6553730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673237344, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d960, mutex=0x7f91ea05d968, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d960, mutex_guard=0x7f91d6522628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d960, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d950) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d950, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d65229a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6522e98, core=0x7f95691792d0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6522e98, core=0x7f95691792d0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6522e98, core=0x7f95691792d0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6524610, t=0x7f91d6522e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d65245d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6522e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6523098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6523220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b228) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b228, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b220, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6523398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b220, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d65235b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=52) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6523a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1246 (Thread 0x7f91d6758640 (LWP 2741248) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6754750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6754750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6754730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673237152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d8a0, mutex=0x7f91ea05d8a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d8a0, mutex_guard=0x7f91d6723628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d8a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d890) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d890, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d67239a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6723e98, core=0x7f9569179230, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6723e98, core=0x7f9569179230) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6723e98, core=0x7f9569179230) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6725610, t=0x7f91d6723e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d67255d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6723e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6724098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6724220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b128) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b128, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b120, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6724398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b120, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d67245b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=51) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6724a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1245 (Thread 0x7f91d6959640 (LWP 2741247) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6955750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6955750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6955730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673237056, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d840, mutex=0x7f91ea05d848, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d840, mutex_guard=0x7f91d6924628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d840, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d830) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d830, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d69249a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6924e98, core=0x7f95691791e0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6924e98, core=0x7f95691791e0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6924e98, core=0x7f95691791e0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6926610, t=0x7f91d6924e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d69265d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6924e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6925098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6925220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b0a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b0a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b0a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6925398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b0a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d69255b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=50) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6925a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1244 (Thread 0x7f91d6b5a640 (LWP 2741246) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6b56750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6b56750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6b56730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236960, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d7e0, mutex=0x7f91ea05d7e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d7e0, mutex_guard=0x7f91d6b25628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d7e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d7d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d7d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d6b259a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6b25e98, core=0x7f9569179190, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6b25e98, core=0x7f9569179190) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6b25e98, core=0x7f9569179190) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6b27610, t=0x7f91d6b25e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d6b275d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6b25e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6b26098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6b26220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f91ea20b028) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f91ea20b028, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91ea20b020, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6b26398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f91ea20b020, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d6b265b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=49) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6b26a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1243 (Thread 0x7f91d6d5b640 (LWP 2741245) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6d57750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6d57750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6d57730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236864, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d780, mutex=0x7f91ea05d788, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d780, mutex_guard=0x7f91d6d26628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d780, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d770) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d770, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d6d269a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6d26e98, core=0x7f9569179140, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6d26e98, core=0x7f9569179140) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6d26e98, core=0x7f9569179140) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6d28610, t=0x7f91d6d26e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d6d285d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6d26e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6d27098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6d27220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b55a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b55a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b55a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6d27398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b55a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d6d275b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=48) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6d27a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1242 (Thread 0x7f91d6f5c640 (LWP 2741244) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d6f58750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d6f58750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d6f58730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236768, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d720, mutex=0x7f91ea05d728, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d720, mutex_guard=0x7f91d6f27628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d720, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d710) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d710, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d6f279a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d6f27e98, core=0x7f95691790f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d6f27e98, core=0x7f95691790f0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d6f27e98, core=0x7f95691790f0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d6f29610, t=0x7f91d6f27e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d6f295d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d6f27e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d6f28098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d6f28220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c9a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c9a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c9a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d6f28398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c9a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d6f285b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=47) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d6f28a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1241 (Thread 0x7f91d715d640 (LWP 2741243) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d7159750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d7159750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d7159730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236672, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d6c0, mutex=0x7f91ea05d6c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d6c0, mutex_guard=0x7f91d7128628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d6c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d6b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d6b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d71289a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d7128e98, core=0x7f95691790a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d7128e98, core=0x7f95691790a0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d7128e98, core=0x7f95691790a0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d712a610, t=0x7f91d7128e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d712a5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d7128e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d7129098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d7129220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b5528) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b5528, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b5520, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d7129398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b5520, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d71295b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=46) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d7129a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1240 (Thread 0x7f91d735e640 (LWP 2741242) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d735a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d735a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d735a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236576, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d660, mutex=0x7f91ea05d668, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d660, mutex_guard=0x7f91d7329628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d660, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d650) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d650, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d73299a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d7329e98, core=0x7f9569179050, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d7329e98, core=0x7f9569179050) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d7329e98, core=0x7f9569179050) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d732b610, t=0x7f91d7329e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d732b5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d7329e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d732a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d732a220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b54a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b54a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b54a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d732a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b54a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d732a5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=45) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d732aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1239 (Thread 0x7f91d755f640 (LWP 2741241) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d755b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d755b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d755b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236480, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d600, mutex=0x7f91ea05d608, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d600, mutex_guard=0x7f91d752a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d600, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d5f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d5f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d752a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d752ae98, core=0x7f9569179000, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d752ae98, core=0x7f9569179000) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d752ae98, core=0x7f9569179000) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d752c610, t=0x7f91d752ae90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d752c5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d752ae90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d752b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d752b220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b5428) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b5428, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b5420, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d752b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b5420, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d752b5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=44) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d752ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1238 (Thread 0x7f91d7760640 (LWP 2741240) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d775c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d775c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d775c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236384, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d5a0, mutex=0x7f91ea05d5a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d5a0, mutex_guard=0x7f91d772b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d5a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d590) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d590, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d772b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d772be98, core=0x7f956a845ec0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d772be98, core=0x7f956a845ec0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d772be98, core=0x7f956a845ec0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d772d610, t=0x7f91d772be90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d772d5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d772be90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d772c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d772c220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b53a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b53a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b53a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d772c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b53a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d772c5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=43) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d772ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1237 (Thread 0x7f91d7961640 (LWP 2741239) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d795d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d795d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d795d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236192, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d4e0, mutex=0x7f91ea05d4e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d4e0, mutex_guard=0x7f91d792c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d4e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d4d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d4d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d792c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d792ce98, core=0x7f956a845d80, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d792ce98, core=0x7f956a845d80) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d792ce98, core=0x7f956a845d80) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d792e610, t=0x7f91d792ce90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d792e5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d792ce90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d792d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d792d220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615948a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615948a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615948a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d792d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615948a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d792d5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=42) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d792da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1236 (Thread 0x7f91d7b62640 (LWP 2741238) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d7b5e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d7b5e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d7b5e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236288, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d540, mutex=0x7f91ea05d548, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d540, mutex_guard=0x7f91d7b2d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d540, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d530) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d530, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d7b2d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d7b2de98, core=0x7f956a845dd0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d7b2de98, core=0x7f956a845dd0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d7b2de98, core=0x7f956a845dd0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d7b2f610, t=0x7f91d7b2de90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d7b2f5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d7b2de90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d7b2e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d7b2e220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615b5328) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615b5328, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615b5320, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d7b2e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615b5320, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d7b2e5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=41) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d7b2ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1235 (Thread 0x7f91d7d63640 (LWP 2741237) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d7d5f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d7d5f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d7d5f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236096, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d480, mutex=0x7f91ea05d488, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d480, mutex_guard=0x7f91d7d2e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d480, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d470) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d470, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d7d2e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d7d2ee98, core=0x7f956a845d30, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d7d2ee98, core=0x7f956a845d30) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d7d2ee98, core=0x7f956a845d30) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d7d30610, t=0x7f91d7d2ee90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d7d305d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d7d2ee90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d7d2f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d7d2f220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594828) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594828, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594820, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d7d2f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594820, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d7d2f5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=40) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d7d2fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1234 (Thread 0x7f91d7f64640 (LWP 2741236) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d7f60750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d7f60750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d7f60730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673236000, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d420, mutex=0x7f91ea05d428, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d420, mutex_guard=0x7f91d7f2f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d420, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d410) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d410, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d7f2f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d7f2fe98, core=0x7f956a845bf0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d7f2fe98, core=0x7f956a845bf0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d7f2fe98, core=0x7f956a845bf0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d7f31610, t=0x7f91d7f2fe90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d7f315d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d7f2fe90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d7f30098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d7f30220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615947a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615947a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615947a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d7f30398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615947a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d7f305b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=39) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d7f30a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1233 (Thread 0x7f91d8165640 (LWP 2741235) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8161750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8161750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8161730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673235904, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d3c0, mutex=0x7f91ea05d3c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d3c0, mutex_guard=0x7f91d8130628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d3c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d3b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d3b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d81309a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8130e98, core=0x7f956a845b00, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8130e98, core=0x7f956a845b00) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8130e98, core=0x7f956a845b00) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8132610, t=0x7f91d8130e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d81325d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8130e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8131098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8131220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594728) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594728, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594720, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8131398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594720, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d81315b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=38) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8131a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1232 (Thread 0x7f91d8366640 (LWP 2741234) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8362750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8362750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8362730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673235808, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d360, mutex=0x7f91ea05d368, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d360, mutex_guard=0x7f91d8331628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d360, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d350) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d350, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d83319a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8331e98, core=0x7f956a845a60, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8331e98, core=0x7f956a845a60) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8331e98, core=0x7f956a845a60) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8333610, t=0x7f91d8331e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d83335d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8331e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8332098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8332220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615946a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615946a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615946a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8332398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615946a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d83325b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=37) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8332a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1231 (Thread 0x7f91d8567640 (LWP 2741233) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8563750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8563750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8563730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673235712, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d300, mutex=0x7f91ea05d308, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d300, mutex_guard=0x7f91d8532628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d300, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d2f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d2f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d85329a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8532e98, core=0x7f956a845a10, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8532e98, core=0x7f956a845a10) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8532e98, core=0x7f956a845a10) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8534610, t=0x7f91d8532e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d85345d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8532e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8533098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8533220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594628) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594628, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594620, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8533398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594620, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d85335b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=36) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8533a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1230 (Thread 0x7f91d8768640 (LWP 2741232) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8764750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8764750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8764730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673235616, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d2a0, mutex=0x7f91ea05d2a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d2a0, mutex_guard=0x7f91d8733628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d2a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d290) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d290, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d87339a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8733e98, core=0x7f956a8459c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8733e98, core=0x7f956a8459c0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8733e98, core=0x7f956a8459c0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8735610, t=0x7f91d8733e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d87355d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8733e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8734098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8734220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615945a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615945a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615945a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8734398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615945a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d87345b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=35) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8734a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1229 (Thread 0x7f91d8969640 (LWP 2741231) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8965750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8965750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8965730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673235520, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d240, mutex=0x7f91ea05d248, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d240, mutex_guard=0x7f91d8934628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d240, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d230) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d230, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d89349a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8934e98, core=0x7f956a845970, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8934e98, core=0x7f956a845970) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8934e98, core=0x7f956a845970) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8936610, t=0x7f91d8934e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d89365d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8934e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8935098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8935220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594528) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594528, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594520, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8935398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594520, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d89355b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=34) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8935a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1228 (Thread 0x7f91d8b6a640 (LWP 2741230) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8b66750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8b66750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8b66730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673235424, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d1e0, mutex=0x7f91ea05d1e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d1e0, mutex_guard=0x7f91d8b35628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d1e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d1d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d1d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d8b359a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8b35e98, core=0x7f956a845920, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8b35e98, core=0x7f956a845920) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8b35e98, core=0x7f956a845920) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8b37610, t=0x7f91d8b35e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d8b375d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8b35e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8b36098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8b36220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615944a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615944a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615944a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8b36398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615944a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d8b365b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=33) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8b36a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1227 (Thread 0x7f91d8d6b640 (LWP 2741229) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8d67750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8d67750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8d67730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673235328, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d180, mutex=0x7f91ea05d188, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d180, mutex_guard=0x7f91d8d36628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d180, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d170) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d170, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d8d369a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8d36e98, core=0x7f956a8458d0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8d36e98, core=0x7f956a8458d0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8d36e98, core=0x7f956a8458d0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8d38610, t=0x7f91d8d36e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d8d385d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8d36e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8d37098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8d37220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594428) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594428, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594420, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8d37398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594420, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d8d375b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=32) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8d37a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1226 (Thread 0x7f91d8f6c640 (LWP 2741228) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d8f68750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d8f68750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d8f68730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673235232, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d120, mutex=0x7f91ea05d128, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d120, mutex_guard=0x7f91d8f37628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d120, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d110) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d110, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d8f379a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d8f37e98, core=0x7f956a845880, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d8f37e98, core=0x7f956a845880) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d8f37e98, core=0x7f956a845880) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d8f39610, t=0x7f91d8f37e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d8f395d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d8f37e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d8f38098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d8f38220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615943a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615943a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615943a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d8f38398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615943a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d8f385b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=31) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d8f38a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1225 (Thread 0x7f91d916d640 (LWP 2741227) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d9169750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d9169750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d9169730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673235136, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d0c0, mutex=0x7f91ea05d0c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d0c0, mutex_guard=0x7f91d9138628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d0c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d0b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d0b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d91389a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d9138e98, core=0x7f956a845830, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d9138e98, core=0x7f956a845830) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d9138e98, core=0x7f956a845830) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d913a610, t=0x7f91d9138e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d913a5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d9138e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d9139098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d9139220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594328) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594328, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594320, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d9139398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594320, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d91395b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=30) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d9139a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1224 (Thread 0x7f91d936e640 (LWP 2741226) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d936a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d936a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d936a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673235040, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d060, mutex=0x7f91ea05d068, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d060, mutex_guard=0x7f91d9339628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d060, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05d050) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05d050, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d93399a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d9339e98, core=0x7f956a8457e0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d9339e98, core=0x7f956a8457e0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d9339e98, core=0x7f956a8457e0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d933b610, t=0x7f91d9339e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d933b5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d9339e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d933a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d933a220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615942a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615942a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615942a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d933a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615942a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d933a5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=29) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d933aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1223 (Thread 0x7f91d956f640 (LWP 2741225) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d956b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d956b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d956b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673234944, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05d000, mutex=0x7f91ea05d008, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05d000, mutex_guard=0x7f91d953a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05d000, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cff0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cff0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d953a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d953ae98, core=0x7f956a845790, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d953ae98, core=0x7f956a845790) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d953ae98, core=0x7f956a845790) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d953c610, t=0x7f91d953ae90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d953c5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d953ae90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d953b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d953b220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507928) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507928, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507920, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d953b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507920, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d953b5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=28) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d953ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1222 (Thread 0x7f91d9770640 (LWP 2741224) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d976c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d976c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d976c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673234848, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cfa0, mutex=0x7f91ea05cfa8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cfa0, mutex_guard=0x7f91d973b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cfa0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cf90) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cf90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d973b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d973be98, core=0x7f956a845740, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d973be98, core=0x7f956a845740) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d973be98, core=0x7f956a845740) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d973d610, t=0x7f91d973be90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d973d5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d973be90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d973c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d973c220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615078a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615078a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615078a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d973c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615078a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d973c5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=27) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d973ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1221 (Thread 0x7f91d9971640 (LWP 2741223) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d996d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d996d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d996d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673234752, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cf40, mutex=0x7f91ea05cf48, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cf40, mutex_guard=0x7f91d993c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cf40, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cf30) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cf30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d993c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d993ce98, core=0x7f956a8456a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d993ce98, core=0x7f956a8456a0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d993ce98, core=0x7f956a8456a0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d993e610, t=0x7f91d993ce90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d993e5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d993ce90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d993d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d993d220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507828) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507828, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507820, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d993d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507820, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d993d5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=26) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d993da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1220 (Thread 0x7f91d9b72640 (LWP 2741222) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d9b6e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d9b6e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d9b6e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673234656, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cee0, mutex=0x7f91ea05cee8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cee0, mutex_guard=0x7f91d9b3d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cee0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ced0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ced0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d9b3d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d9b3de98, core=0x7f956a8452e0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d9b3de98, core=0x7f956a8452e0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d9b3de98, core=0x7f956a8452e0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d9b3f610, t=0x7f91d9b3de90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d9b3f5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d9b3de90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d9b3e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d9b3e220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615077a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615077a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615077a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d9b3e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615077a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d9b3e5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=25) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d9b3ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1219 (Thread 0x7f91d9d73640 (LWP 2741221) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d9d6f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d9d6f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d9d6f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673234560, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05ce80, mutex=0x7f91ea05ce88, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05ce80, mutex_guard=0x7f91d9d3e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05ce80, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ce70) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ce70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d9d3e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d9d3ee98, core=0x7f956a845290, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d9d3ee98, core=0x7f956a845290) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d9d3ee98, core=0x7f956a845290) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d9d40610, t=0x7f91d9d3ee90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d9d405d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d9d3ee90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d9d3f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d9d3f220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507728) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507728, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507720, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d9d3f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507720, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d9d3f5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=24) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d9d3fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1218 (Thread 0x7f91d9f74640 (LWP 2741220) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91d9f70750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91d9f70750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91d9f70730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673234464, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05ce20, mutex=0x7f91ea05ce28, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05ce20, mutex_guard=0x7f91d9f3f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05ce20, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ce10) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ce10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91d9f3f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91d9f3fe98, core=0x7f956a845240, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91d9f3fe98, core=0x7f956a845240) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91d9f3fe98, core=0x7f956a845240) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91d9f41610, t=0x7f91d9f3fe90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91d9f415d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91d9f3fe90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91d9f40098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91d9f40220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615076a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615076a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615076a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91d9f40398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615076a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91d9f405b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=23) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91d9f40a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1217 (Thread 0x7f91da175640 (LWP 2741219) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91da171750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91da171750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91da171730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673234368, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cdc0, mutex=0x7f91ea05cdc8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cdc0, mutex_guard=0x7f91da140628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cdc0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cdb0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cdb0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91da1409a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91da140e98, core=0x7f956a8451f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91da140e98, core=0x7f956a8451f0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91da140e98, core=0x7f956a8451f0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91da142610, t=0x7f91da140e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91da1425d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91da140e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91da141098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91da141220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507628) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507628, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507620, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91da141398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507620, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91da1415b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=22) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91da141a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1216 (Thread 0x7f91da376640 (LWP 2741218) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91da372750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91da372750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91da372730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673234272, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cd60, mutex=0x7f91ea05cd68, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cd60, mutex_guard=0x7f91da341628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cd60, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cd50) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cd50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91da3419a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91da341e98, core=0x7f956a844f70, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91da341e98, core=0x7f956a844f70) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91da341e98, core=0x7f956a844f70) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91da343610, t=0x7f91da341e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91da3435d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91da341e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91da342098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91da342220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615075a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615075a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615075a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91da342398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615075a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91da3425b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=21) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91da342a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1215 (Thread 0x7f91da577640 (LWP 2741217) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91da573750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91da573750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91da573730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673234176, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cd00, mutex=0x7f91ea05cd08, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cd00, mutex_guard=0x7f91da542628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cd00, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ccf0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ccf0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91da5429a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91da542e98, core=0x7f956a8437b0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91da542e98, core=0x7f956a8437b0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91da542e98, core=0x7f956a8437b0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91da544610, t=0x7f91da542e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91da5445d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91da542e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91da543098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91da543220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507528) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507528, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507520, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91da543398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507520, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91da5435b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=20) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91da543a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1214 (Thread 0x7f91da778640 (LWP 2741216) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91da774750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91da774750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91da774730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673234080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cca0, mutex=0x7f91ea05cca8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cca0, mutex_guard=0x7f91da743628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cca0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cc90) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cc90, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91da7439a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91da743e98, core=0x7f956a843760, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91da743e98, core=0x7f956a843760) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91da743e98, core=0x7f956a843760) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91da745610, t=0x7f91da743e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91da7455d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91da743e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91da744098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91da744220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615074a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615074a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615074a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91da744398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615074a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91da7445b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=19) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91da744a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1213 (Thread 0x7f91da979640 (LWP 2741215) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91da975750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91da975750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91da975730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233984, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cc40, mutex=0x7f91ea05cc48, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cc40, mutex_guard=0x7f91da944628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cc40, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cc30) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cc30, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91da9449a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91da944e98, core=0x7f956a8433f0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91da944e98, core=0x7f956a8433f0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91da944e98, core=0x7f956a8433f0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91da946610, t=0x7f91da944e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91da9465d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91da944e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91da945098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91da945220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507428) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507428, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507420, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91da945398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507420, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91da9455b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=18) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91da945a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1212 (Thread 0x7f91dab7a640 (LWP 2741214) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dab76750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dab76750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dab76730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233888, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cbe0, mutex=0x7f91ea05cbe8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cbe0, mutex_guard=0x7f91dab45628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cbe0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cbd0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cbd0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dab459a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dab45e98, core=0x7f956a8433a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dab45e98, core=0x7f956a8433a0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dab45e98, core=0x7f956a8433a0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dab47610, t=0x7f91dab45e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dab475d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dab45e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dab46098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dab46220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615073a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615073a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615073a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dab46398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615073a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dab465b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=17) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dab46a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1211 (Thread 0x7f91dad7b640 (LWP 2741213) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dad77750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dad77750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dad77730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233792, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cb80, mutex=0x7f91ea05cb88, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cb80, mutex_guard=0x7f91dad46628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cb80, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cb70) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cb70, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dad469a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dad46e98, core=0x7f956a843350, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dad46e98, core=0x7f956a843350) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dad46e98, core=0x7f956a843350) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dad48610, t=0x7f91dad46e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dad485d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dad46e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dad47098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dad47220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507328) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507328, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507320, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dad47398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507320, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dad475b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=16) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dad47a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1210 (Thread 0x7f91daf7c640 (LWP 2741212) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91daf78750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91daf78750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91daf78730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233696, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cb20, mutex=0x7f91ea05cb28, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cb20, mutex_guard=0x7f91daf47628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cb20, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cb10) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cb10, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91daf479a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91daf47e98, core=0x7f956a843210, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91daf47e98, core=0x7f956a843210) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91daf47e98, core=0x7f956a843210) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91daf49610, t=0x7f91daf47e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91daf495d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91daf47e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91daf48098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91daf48220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615072a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615072a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615072a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91daf48398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615072a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91daf485b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=15) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91daf48a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1209 (Thread 0x7f91db17d640 (LWP 2741211) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91db179750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91db179750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91db179730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233600, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05cac0, mutex=0x7f91ea05cac8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05cac0, mutex_guard=0x7f91db148628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05cac0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05cab0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05cab0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91db1489a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91db148e98, core=0x7f956a8431c0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91db148e98, core=0x7f956a8431c0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91db148e98, core=0x7f956a8431c0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91db14a610, t=0x7f91db148e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91db14a5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91db148e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91db149098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91db149220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507228) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507228, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507220, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91db149398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507220, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91db1495b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=14) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91db149a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1208 (Thread 0x7f91db37e640 (LWP 2741210) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91db37a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91db37a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91db37a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233504, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05ca60, mutex=0x7f91ea05ca68, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05ca60, mutex_guard=0x7f91db349628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05ca60, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05ca50) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05ca50, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91db3499a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91db349e98, core=0x7f956a843170, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91db349e98, core=0x7f956a843170) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91db349e98, core=0x7f956a843170) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91db34b610, t=0x7f91db349e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91db34b5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91db349e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91db34a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91db34a220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615071a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615071a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615071a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91db34a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615071a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91db34a5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=13) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91db34aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1207 (Thread 0x7f91db57f640 (LWP 2741209) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91db57b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91db57b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91db57b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233408, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05ca00, mutex=0x7f91ea05ca08, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05ca00, mutex_guard=0x7f91db54a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05ca00, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c9f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c9f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91db54a9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91db54ae98, core=0x7f956a842fe0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91db54ae98, core=0x7f956a842fe0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91db54ae98, core=0x7f956a842fe0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91db54c610, t=0x7f91db54ae90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91db54c5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91db54ae90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91db54b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91db54b220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507128) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507128, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507120, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91db54b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507120, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91db54b5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=12) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91db54ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1206 (Thread 0x7f91db780640 (LWP 2741208) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91db77c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91db77c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91db77c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233312, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c9a0, mutex=0x7f91ea05c9a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c9a0, mutex_guard=0x7f91db74b628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c9a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c990) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c990, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91db74b9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91db74be98, core=0x7f956a842f90, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91db74be98, core=0x7f956a842f90) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91db74be98, core=0x7f956a842f90) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91db74d610, t=0x7f91db74be90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91db74d5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91db74be90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91db74c098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91db74c220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f95615070a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f95615070a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f95615070a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91db74c398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f95615070a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91db74c5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=11) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91db74ca20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1205 (Thread 0x7f91db981640 (LWP 2741207) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91db97d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91db97d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91db97d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233216, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c940, mutex=0x7f91ea05c948, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c940, mutex_guard=0x7f91db94c628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c940, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c930) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c930, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91db94c9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91db94ce98, core=0x7f956a842ea0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91db94ce98, core=0x7f956a842ea0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91db94ce98, core=0x7f956a842ea0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91db94e610, t=0x7f91db94ce90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91db94e5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91db94ce90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91db94d098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91db94d220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507028) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507028, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507020, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91db94d398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507020, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91db94d5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=10) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91db94da20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1204 (Thread 0x7f91dbb82640 (LWP 2741206) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dbb7e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dbb7e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dbb7e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c8e0, mutex=0x7f91ea05c8e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c8e0, mutex_guard=0x7f91dbb4d628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c8e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c8d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c8d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dbb4d9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dbb4de98, core=0x7f956a842e50, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dbb4de98, core=0x7f956a842e50) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dbb4de98, core=0x7f956a842e50) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dbb4f610, t=0x7f91dbb4de90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dbb4f5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dbb4de90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dbb4e098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dbb4e220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919cfa8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919cfa8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919cfa0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dbb4e398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919cfa0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dbb4e5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=9) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dbb4ea20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1203 (Thread 0x7f91dbd83640 (LWP 2741205) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dbd7f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dbd7f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dbd7f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673233024, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c880, mutex=0x7f91ea05c888, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c880, mutex_guard=0x7f91dbd4e628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c880, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c870) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c870, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dbd4e9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dbd4ee98, core=0x7f956a842e00, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dbd4ee98, core=0x7f956a842e00) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dbd4ee98, core=0x7f956a842e00) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dbd50610, t=0x7f91dbd4ee90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dbd505d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dbd4ee90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dbd4f098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dbd4f220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919cf28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919cf28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919cf20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dbd4f398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919cf20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dbd4f5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=8) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dbd4fa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1202 (Thread 0x7f91dbf84640 (LWP 2741204) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dbf80750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dbf80750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dbf80730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673232928, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c820, mutex=0x7f91ea05c828, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c820, mutex_guard=0x7f91dbf4f628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c820, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c810) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c810, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dbf4f9a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dbf4fe98, core=0x7f956a842db0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dbf4fe98, core=0x7f956a842db0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dbf4fe98, core=0x7f956a842db0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dbf51610, t=0x7f91dbf4fe90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dbf515d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dbf4fe90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dbf50098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dbf50220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919cea8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919cea8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919cea0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dbf50398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919cea0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dbf505b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=7) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dbf50a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1201 (Thread 0x7f91dc185640 (LWP 2741203) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dc181750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dc181750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dc181730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673232832, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c7c0, mutex=0x7f91ea05c7c8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c7c0, mutex_guard=0x7f91dc150628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c7c0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c7b0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c7b0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dc1509a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dc150e98, core=0x7f956a842d60, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dc150e98, core=0x7f956a842d60) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dc150e98, core=0x7f956a842d60) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dc152610, t=0x7f91dc150e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dc1525d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dc150e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dc151098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dc151220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919ce28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919ce28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919ce20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dc151398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919ce20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dc1515b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=6) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dc151a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1200 (Thread 0x7f91dc386640 (LWP 2741202) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dc382750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dc382750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dc382730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673232736, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c760, mutex=0x7f91ea05c768, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c760, mutex_guard=0x7f91dc351628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c760, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c750) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c750, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dc3519a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dc351e98, core=0x7f956a842d10, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dc351e98, core=0x7f956a842d10) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dc351e98, core=0x7f956a842d10) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dc353610, t=0x7f91dc351e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dc3535d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dc351e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dc352098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dc352220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919ca28) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919ca28, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919ca20, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dc352398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919ca20, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dc3525b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=5) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dc352a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1199 (Thread 0x7f91dc587640 (LWP 2741201) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dc583750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dc583750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dc583730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673232640, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c700, mutex=0x7f91ea05c708, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c700, mutex_guard=0x7f91dc552628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c700, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c6f0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c6f0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dc5529a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dc552e98, core=0x7f956a842cc0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dc552e98, core=0x7f956a842cc0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dc552e98, core=0x7f956a842cc0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dc554610, t=0x7f91dc552e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dc5545d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dc552e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dc553098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dc553220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c928) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c928, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c920, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dc553398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c920, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dc5535b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=4) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dc553a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1198 (Thread 0x7f91dc788640 (LWP 2741200) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dc784750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dc784750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dc784730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673232544, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c6a0, mutex=0x7f91ea05c6a8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c6a0, mutex_guard=0x7f91dc753628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c6a0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c690) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c690, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dc7539a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dc753e98, core=0x7f956a842c70, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dc753e98, core=0x7f956a842c70) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dc753e98, core=0x7f956a842c70) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dc755610, t=0x7f91dc753e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dc7555d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dc753e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dc754098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dc754220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c8a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c8a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c8a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dc754398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c8a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dc7545b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=3) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dc754a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1197 (Thread 0x7f91dc989640 (LWP 2741199) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dc985750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dc985750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dc985730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673232448, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c640, mutex=0x7f91ea05c648, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c640, mutex_guard=0x7f91dc954628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c640, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c630) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c630, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dc9549a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dc954e98, core=0x7f956a842c20, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dc954e98, core=0x7f956a842c20) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dc954e98, core=0x7f956a842c20) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dc956610, t=0x7f91dc954e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dc9565d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dc954e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dc955098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dc955220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c828) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c828, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c820, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dc955398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c820, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dc9555b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=2) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dc955a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1196 (Thread 0x7f91dcb8a640 (LWP 2741198) "impwkr-v1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dcb86750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dcb86750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dcb86730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673232352, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c5e0, mutex=0x7f91ea05c5e8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c5e0, mutex_guard=0x7f91dcb55628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c5e0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c5d0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c5d0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dcb559a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dcb55e98, core=0x7f956a842bd0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dcb55e98, core=0x7f956a842bd0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dcb55e98, core=0x7f956a842bd0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dcb57610, t=0x7f91dcb55e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dcb575d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dcb55e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dcb56098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dcb56220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c7a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c7a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c7a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dcb56398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c7a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dcb565b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=1) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dcb56a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1195 (Thread 0x7f91dcd8b640 (LWP 2741197) "impwkr-v1"): -#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 -#1 0x00007f9583185778 in mio::sys::unix::selector::epoll::Selector::select (self=0x7f956919c6b8, events=0x7f956919c698, timeout=...) at src/sys/unix/selector/epoll.rs:97 -#2 0x00007f9583184fd1 in mio::poll::Poll::poll (self=0x7f956919c6b0, events=0x7f956919c698, timeout=...) at src/poll.rs:426 -#3 0x00007f9583167b7c in tokio::runtime::io::driver::Driver::turn (self=0x7f956919c698, handle=0x7f91ea093518, max_wait=...) at src/runtime/io/driver.rs:149 -#4 0x00007f95831677ee in tokio::runtime::io::driver::Driver::park (self=0x7f956919c698, rt_handle=0x7f91ea093518) at src/runtime/io/driver.rs:122 -#5 0x00007f958316a887 in tokio::runtime::signal::Driver::park (self=0x7f956919c698, handle=0x7f91ea093518) at src/runtime/signal/mod.rs:92 -#6 0x00007f9583152787 in tokio::runtime::process::Driver::park (self=0x7f956919c698, handle=0x7f91ea093518) at src/runtime/process.rs:32 -#7 0x00007f95830f1aca in tokio::runtime::driver::IoStack::park (self=0x7f956919c698, handle=0x7f91ea093518) at src/runtime/driver.rs:175 -#8 0x00007f958316a02e in tokio::runtime::time::Driver::park_internal (self=0x7f956919c698, rt_handle=0x7f91ea093518, limit=...) at src/runtime/time/mod.rs:235 -#9 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f956919c698, handle=0x7f91ea093518) at src/runtime/time/mod.rs:165 -#10 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f956919c690, handle=0x7f91ea093518) at src/runtime/driver.rs:332 -#11 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f956919c690, handle=0x7f91ea093518) at src/runtime/driver.rs:71 -#12 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f91ea05c570, driver=0x7f956919c690, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:184 -#13 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c570, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:117 -#14 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dcd569a0, handle=0x7f91ea093518) at src/runtime/scheduler/multi_thread/park.rs:67 -#15 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dcd56e98, core=0x7f956a842b80, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#16 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dcd56e98, core=0x7f956a842b80) at src/runtime/scheduler/multi_thread/worker.rs:702 -#17 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dcd56e98, core=0x7f956a842b80) at src/runtime/scheduler/multi_thread/worker.rs:553 -#18 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#19 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dcd58610, t=0x7f91dcd56e90, f=...) at src/runtime/context/scoped.rs:40 -#20 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dcd585d8) at src/runtime/context.rs:180 -#21 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#22 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#23 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dcd56e90, f=...) at src/runtime/context.rs:180 -#24 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#25 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dcd57098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#26 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#27 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#28 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dcd57220) at src/runtime/blocking/task.rs:42 -#29 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c728) at src/runtime/task/core.rs:328 -#30 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c728, f=...) at src/loom/std/unsafe_cell.rs:16 -#31 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c720, cx=...) at src/runtime/task/core.rs:317 -#32 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#33 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#34 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dcd57398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#35 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#36 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#37 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#38 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c720, cx=...) at src/runtime/task/harness.rs:473 -#39 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dcd575b0) at src/runtime/task/harness.rs:208 -#40 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#41 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#42 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#43 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#44 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#45 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de810, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 -#46 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#47 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#48 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#49 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#50 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dcd57a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#51 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#52 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#53 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#54 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#55 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#56 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#57 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#58 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#59 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#60 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1194 (Thread 0x7f91dcf8c640 (LWP 2741196) "impwkr-v0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dcf88750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dcf88750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dcf88730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673231536, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c2b0, mutex=0x7f91ea05c2b8, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c2b0, mutex_guard=0x7f91dcf57628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c2b0, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c2a0) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c2a0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dcf579a0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dcf57e98, core=0x7f956a842130, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dcf57e98, core=0x7f956a842130) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dcf57e98, core=0x7f956a842130) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dcf59610, t=0x7f91dcf57e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dcf595d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dcf57e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dcf58098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dcf58220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c5a8) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c5a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c5a0, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dcf58398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c5a0, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dcf585b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de590, worker_thread_id=3) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dcf58a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1193 (Thread 0x7f91dd18d640 (LWP 2741195) "impwkr-v0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dd189750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dd189750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dd189730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673231440, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c250, mutex=0x7f91ea05c258, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c250, mutex_guard=0x7f91dd158628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c250, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c240) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c240, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dd1589a0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dd158e98, core=0x7f956a842090, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dd158e98, core=0x7f956a842090) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dd158e98, core=0x7f956a842090) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dd15a610, t=0x7f91dd158e90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dd15a5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dd158e90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dd159098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dd159220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c528) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c528, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c520, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dd159398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c520, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dd1595b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de590, worker_thread_id=2) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dd159a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1192 (Thread 0x7f91dd38e640 (LWP 2741194) "impwkr-v0"): -#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 -#1 0x00007f9583185778 in mio::sys::unix::selector::epoll::Selector::select (self=0x7f956919c3b8, events=0x7f956919c398, timeout=...) at src/sys/unix/selector/epoll.rs:97 -#2 0x00007f9583184fd1 in mio::poll::Poll::poll (self=0x7f956919c3b0, events=0x7f956919c398, timeout=...) at src/poll.rs:426 -#3 0x00007f9583167b7c in tokio::runtime::io::driver::Driver::turn (self=0x7f956919c398, handle=0x7f91ea093318, max_wait=...) at src/runtime/io/driver.rs:149 -#4 0x00007f958316786a in tokio::runtime::io::driver::Driver::park_timeout (self=0x7f956919c398, rt_handle=0x7f91ea093318, duration=...) at src/runtime/io/driver.rs:127 -#5 0x00007f958316a8c0 in tokio::runtime::signal::Driver::park_timeout (self=0x7f956919c398, handle=0x7f91ea093318, duration=...) at src/runtime/signal/mod.rs:97 -#6 0x00007f95831527c0 in tokio::runtime::process::Driver::park_timeout (self=0x7f956919c398, handle=0x7f91ea093318, duration=...) at src/runtime/process.rs:37 -#7 0x00007f95830f1b55 in tokio::runtime::driver::IoStack::park_timeout (self=0x7f956919c398, handle=0x7f91ea093318, duration=...) at src/runtime/driver.rs:182 -#8 0x00007f958316a49c in tokio::runtime::time::Driver::park_thread_timeout (self=0x7f956919c398, rt_handle=0x7f91ea093318, duration=...) at src/runtime/time/mod.rs:270 -#9 0x00007f958316a1f5 in tokio::runtime::time::Driver::park_internal (self=0x7f956919c398, rt_handle=0x7f91ea093318, limit=...) at src/runtime/time/mod.rs:226 -#10 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f956919c398, handle=0x7f91ea093318) at src/runtime/time/mod.rs:165 -#11 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f956919c390, handle=0x7f91ea093318) at src/runtime/driver.rs:332 -#12 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f956919c390, handle=0x7f91ea093318) at src/runtime/driver.rs:71 -#13 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f91ea05c1e0, driver=0x7f956919c390, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:184 -#14 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c1e0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:117 -#15 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dd3599a0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:67 -#16 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dd359e98, core=0x7f956a842040, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#17 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dd359e98, core=0x7f956a842040) at src/runtime/scheduler/multi_thread/worker.rs:702 -#18 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dd359e98, core=0x7f956a842040) at src/runtime/scheduler/multi_thread/worker.rs:553 -#19 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#20 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dd35b610, t=0x7f91dd359e90, f=...) at src/runtime/context/scoped.rs:40 -#21 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dd35b5d8) at src/runtime/context.rs:180 -#22 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#23 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#24 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dd359e90, f=...) at src/runtime/context.rs:180 -#25 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#26 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dd35a098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#27 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#28 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#29 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dd35a220) at src/runtime/blocking/task.rs:42 -#30 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c4a8) at src/runtime/task/core.rs:328 -#31 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c4a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#32 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c4a0, cx=...) at src/runtime/task/core.rs:317 -#33 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#34 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#35 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dd35a398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#36 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#37 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#38 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#39 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c4a0, cx=...) at src/runtime/task/harness.rs:473 -#40 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dd35a5b0) at src/runtime/task/harness.rs:208 -#41 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#42 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#43 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#44 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#45 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#46 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de590, worker_thread_id=1) at src/runtime/blocking/pool.rs:513 -#47 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#48 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#49 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#50 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#51 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dd35aa20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#52 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#53 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#54 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#55 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#56 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#57 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#58 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#59 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#60 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#61 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1191 (Thread 0x7f91dd58f640 (LWP 2741193) "impwkr-v0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91dd58b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91dd58b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91dd58b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140264673231248, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f91ea05c190, mutex=0x7f91ea05c198, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f91ea05c190, mutex_guard=0x7f91dd55a628) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f91ea05c190, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830f2d44 in tokio::runtime::scheduler::multi_thread::park::Inner::park_condvar (self=0x7f91ea05c180) at src/runtime/scheduler/multi_thread/park.rs:148 -#10 0x00007f95830f2b64 in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c180, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:119 -#11 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91dd55a9a0, handle=0x7f91ea093318) at src/runtime/scheduler/multi_thread/park.rs:67 -#12 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91dd55ae98, core=0x7f956a841ff0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#13 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91dd55ae98, core=0x7f956a841ff0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#14 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91dd55ae98, core=0x7f956a841ff0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#15 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#16 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91dd55c610, t=0x7f91dd55ae90, f=...) at src/runtime/context/scoped.rs:40 -#17 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91dd55c5d8) at src/runtime/context.rs:180 -#18 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#19 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#20 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91dd55ae90, f=...) at src/runtime/context.rs:180 -#21 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#22 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91dd55b098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#23 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#24 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#25 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91dd55b220) at src/runtime/blocking/task.rs:42 -#26 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c428) at src/runtime/task/core.rs:328 -#27 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c428, f=...) at src/loom/std/unsafe_cell.rs:16 -#28 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c420, cx=...) at src/runtime/task/core.rs:317 -#29 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#30 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#31 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91dd55b398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#32 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#33 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#34 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#35 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c420, cx=...) at src/runtime/task/harness.rs:473 -#36 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91dd55b5b0) at src/runtime/task/harness.rs:208 -#37 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#38 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#39 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#40 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#41 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#42 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de590, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 -#43 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#44 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#45 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#46 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#47 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91dd55ba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#48 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#49 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#50 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#51 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#52 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#53 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#54 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#55 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#56 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#57 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1190 (Thread 0x7f94c19ff640 (LWP 2741192) "rtkp"): -#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 -#1 0x00007f9583185778 in mio::sys::unix::selector::epoll::Selector::select (self=0x7f956919c238, events=0x7f956919c218, timeout=...) at src/sys/unix/selector/epoll.rs:97 -#2 0x00007f9583184fd1 in mio::poll::Poll::poll (self=0x7f956919c230, events=0x7f956919c218, timeout=...) at src/poll.rs:426 -#3 0x00007f9583167b7c in tokio::runtime::io::driver::Driver::turn (self=0x7f956919c218, handle=0x7f91ea093118, max_wait=...) at src/runtime/io/driver.rs:149 -#4 0x00007f95831677ee in tokio::runtime::io::driver::Driver::park (self=0x7f956919c218, rt_handle=0x7f91ea093118) at src/runtime/io/driver.rs:122 -#5 0x00007f958316a887 in tokio::runtime::signal::Driver::park (self=0x7f956919c218, handle=0x7f91ea093118) at src/runtime/signal/mod.rs:92 -#6 0x00007f9583152787 in tokio::runtime::process::Driver::park (self=0x7f956919c218, handle=0x7f91ea093118) at src/runtime/process.rs:32 -#7 0x00007f95830f1aca in tokio::runtime::driver::IoStack::park (self=0x7f956919c218, handle=0x7f91ea093118) at src/runtime/driver.rs:175 -#8 0x00007f958316a02e in tokio::runtime::time::Driver::park_internal (self=0x7f956919c218, rt_handle=0x7f91ea093118, limit=...) at src/runtime/time/mod.rs:235 -#9 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f956919c218, handle=0x7f91ea093118) at src/runtime/time/mod.rs:165 -#10 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f956919c210, handle=0x7f91ea093118) at src/runtime/driver.rs:332 -#11 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f956919c210, handle=0x7f91ea093118) at src/runtime/driver.rs:71 -#12 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f91ea05c060, driver=0x7f956919c210, handle=0x7f91ea093118) at src/runtime/scheduler/multi_thread/park.rs:184 -#13 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f91ea05c060, handle=0x7f91ea093118) at src/runtime/scheduler/multi_thread/park.rs:117 -#14 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f94c19ca9a0, handle=0x7f91ea093118) at src/runtime/scheduler/multi_thread/park.rs:67 -#15 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f94c19cae98, core=0x7f956a841e10, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#16 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f94c19cae98, core=0x7f956a841e10) at src/runtime/scheduler/multi_thread/worker.rs:702 -#17 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f94c19cae98, core=0x7f956a841e10) at src/runtime/scheduler/multi_thread/worker.rs:553 -#18 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#19 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f94c19cc610, t=0x7f94c19cae90, f=...) at src/runtime/context/scoped.rs:40 -#20 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f94c19cc5d8) at src/runtime/context.rs:180 -#21 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#22 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#23 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f94c19cae90, f=...) at src/runtime/context.rs:180 -#24 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#25 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f94c19cb098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#26 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#27 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#28 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f94c19cb220) at src/runtime/blocking/task.rs:42 -#29 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f956919c2a8) at src/runtime/task/core.rs:328 -#30 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f956919c2a8, f=...) at src/loom/std/unsafe_cell.rs:16 -#31 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f956919c2a0, cx=...) at src/runtime/task/core.rs:317 -#32 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#33 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#34 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f94c19cb398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#35 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#36 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#37 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#38 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f956919c2a0, cx=...) at src/runtime/task/harness.rs:473 -#39 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f94c19cb5b0) at src/runtime/task/harness.rs:208 -#40 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#41 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#42 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#43 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#44 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#45 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f92132de450, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 -#46 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#47 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#48 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#49 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#50 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f94c19cba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#51 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#52 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#53 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#54 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#55 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#56 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#57 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#58 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#59 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#60 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1189 (Thread 0x7f91dda10640 (LWP 2741191) "stats-monitor"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cfc2f4 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 0x00007f9583cea9cc in std::sys_common::thread_parking::futex::Parker::park_timeout () at library/std/src/sys_common/thread_parking/futex.rs:72 -#3 std::thread::park_timeout () at library/std/src/thread/mod.rs:1133 -#4 0x00007f957e13f3e0 in std::sync::mpmc::context::Context::wait_until (self=0x7f91dd9dae08, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:130 -#5 0x00007f957e42fc0a in std::sync::mpmc::list::{impl#3}::recv::{closure#1} (cx=0x7f91dd9dae08) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:444 -#6 0x00007f957e432120 in std::sync::mpmc::context::{impl#0}::with::{closure#0}, ()> (cx=, cx=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:50 -#7 std::sync::mpmc::context::{impl#0}::with::{closure#1}, ()> (cell=0x7f91dda0ca38) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:58 -#8 0x00007f957e43c74b in std::thread::local::LocalKey>>::try_with>, std::sync::mpmc::context::{impl#0}::with::{closure_env#1}, ()>, ()> (self=0x7f9583da0378, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957e43194e in std::sync::mpmc::context::Context::with, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:53 -#10 0x00007f957e42f6c9 in std::sync::mpmc::list::Channel::recv (self=0x7f92139a5a00, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:434 -#11 0x00007f957e13ea38 in std::sync::mpmc::Receiver::recv_deadline (self=0x7f91dd9db840, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/mod.rs:340 -#12 0x00007f957e13e7f1 in std::sync::mpmc::Receiver::recv_timeout (self=0x7f91dd9db840, timeout=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/mod.rs:323 -#13 0x00007f957dfe1a9a in std::sync::mpsc::Receiver::recv_timeout (self=0x7f91dd9db840, timeout=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpsc/mod.rs:909 -#14 0x00007f957b5ede37 in raftstore::store::worker::pd::{impl#12}::start::{closure#0}> () at components/raftstore/src/store/worker/pd.rs:713 -#15 0x00007f957df41a59 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#16 0x00007f957a80aef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#17 0x00007f957cb7a6d0 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#18 0x00007f957c9e4d54 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#19 0x00007f957be0a804 in std::panicking::try::do_call>, ()>, ()>>, ()> (data=0x7f91dd9dbed0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#20 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#21 0x00007f957bd876a8 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#22 0x00007f957cb77db0 in std::panic::catch_unwind>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#23 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#24 0x00007f957d67d7af in core::ops::function::FnOnce::call_once>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#25 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#26 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#28 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#29 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1188 (Thread 0x7f91ddc11640 (LWP 2741190) "apply-low-71"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91ddbd6288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91ddbd6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91ddbd6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91ddc0d798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91ddbd7868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91ddbd7868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91ddbd6ee0, batch=0x7f91ddbd6960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91ddbd6ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91ddbd9690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1187 (Thread 0x7f91de092640 (LWP 2741189) "apply-low-70"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91de057288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91de057288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91de057288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91de08e798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de058868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de058868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de057ee0, batch=0x7f91de057960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de057ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91de05a690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1186 (Thread 0x7f91de293640 (LWP 2741188) "apply-low-69"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91de258288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91de258288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91de258288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91de28f798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de259868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de259868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de258ee0, batch=0x7f91de258960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de258ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91de25b690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1185 (Thread 0x7f91de494640 (LWP 2741187) "apply-low-68"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91de459288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91de459288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91de459288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91de490798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de45a868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de45a868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de459ee0, batch=0x7f91de459960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de459ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91de45c690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1184 (Thread 0x7f91de695640 (LWP 2741186) "apply-low-67"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91de65a288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91de65a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91de65a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91de691798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de65b868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de65b868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de65aee0, batch=0x7f91de65a960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de65aee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91de65d690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1183 (Thread 0x7f91de896640 (LWP 2741185) "apply-low-66"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91de85b288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91de85b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91de85b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91de892798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de85c868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91de85c868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de85bee0, batch=0x7f91de85b960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91de85bee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91de85e690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1182 (Thread 0x7f91dea97640 (LWP 2741184) "apply-low-65"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dea5c288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dea5c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dea5c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dea93798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dea5d868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dea5d868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dea5cee0, batch=0x7f91dea5c960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dea5cee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dea5f690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1181 (Thread 0x7f91dec98640 (LWP 2741183) "apply-low-64"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dec5d288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dec5d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dec5d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dec94798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dec5e868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dec5e868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dec5dee0, batch=0x7f91dec5d960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dec5dee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dec60690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1180 (Thread 0x7f91dee99640 (LWP 2741182) "apply-low-63"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dee5e288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dee5e288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dee5e288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dee95798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dee5f868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dee5f868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dee5eee0, batch=0x7f91dee5e960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dee5eee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dee61690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1179 (Thread 0x7f91df09a640 (LWP 2741181) "apply-low-62"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91df05f288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91df05f288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91df05f288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91df096798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df060868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df060868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df05fee0, batch=0x7f91df05f960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df05fee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91df062690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1178 (Thread 0x7f91df29b640 (LWP 2741180) "apply-low-61"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91df260288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91df260288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91df260288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91df297798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df261868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df261868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df260ee0, batch=0x7f91df260960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df260ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91df263690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1177 (Thread 0x7f91df49c640 (LWP 2741179) "apply-low-60"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91df461288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91df461288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91df461288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91df498798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df462868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df462868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df461ee0, batch=0x7f91df461960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df461ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91df464690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1176 (Thread 0x7f91df91d640 (LWP 2741178) "apply-low-59"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91df8e2288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91df8e2288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91df8e2288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91df919798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df8e3868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91df8e3868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df8e2ee0, batch=0x7f91df8e2960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91df8e2ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91df8e5690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1175 (Thread 0x7f91dfb1e640 (LWP 2741177) "apply-low-58"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dfae3288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dfae3288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dfae3288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dfb1a798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfae4868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfae4868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfae3ee0, batch=0x7f91dfae3960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfae3ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dfae6690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1174 (Thread 0x7f91dfd1f640 (LWP 2741176) "apply-low-57"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dfce4288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dfce4288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dfce4288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dfd1b798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfce5868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfce5868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfce4ee0, batch=0x7f91dfce4960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfce4ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dfce7690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1173 (Thread 0x7f91dff20640 (LWP 2741175) "apply-low-56"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91dfee5288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91dfee5288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91dfee5288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91dff1c798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfee6868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91dfee6868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfee5ee0, batch=0x7f91dfee5960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91dfee5ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91dfee8690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1172 (Thread 0x7f91e0121640 (LWP 2741174) "apply-low-55"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e00e6288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e00e6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e00e6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e011d798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e00e7868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e00e7868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e00e6ee0, batch=0x7f91e00e6960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e00e6ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e00e9690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1171 (Thread 0x7f91e0322640 (LWP 2741173) "apply-low-54"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e02e7288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e02e7288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e02e7288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e031e798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e02e8868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e02e8868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e02e7ee0, batch=0x7f91e02e7960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e02e7ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e02ea690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1170 (Thread 0x7f91e0523640 (LWP 2741172) "apply-low-53"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e04e8288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e04e8288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e04e8288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e051f798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e04e9868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e04e9868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e04e8ee0, batch=0x7f91e04e8960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e04e8ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e04eb690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1169 (Thread 0x7f91e0724640 (LWP 2741171) "apply-low-52"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e06e9288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e06e9288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e06e9288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e0720798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e06ea868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e06ea868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e06e9ee0, batch=0x7f91e06e9960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e06e9ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e06ec690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1168 (Thread 0x7f91e0925640 (LWP 2741170) "apply-low-51"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e08ea288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e08ea288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e08ea288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e0921798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e08eb868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e08eb868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e08eaee0, batch=0x7f91e08ea960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e08eaee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e08ed690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1167 (Thread 0x7f91e0b26640 (LWP 2741169) "apply-low-50"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e0aeb288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e0aeb288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e0aeb288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e0b22798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0aec868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0aec868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0aebee0, batch=0x7f91e0aeb960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0aebee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e0aee690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1166 (Thread 0x7f91e0d27640 (LWP 2741168) "apply-low-49"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e0cec288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e0cec288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e0cec288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e0d23798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0ced868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0ced868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0cecee0, batch=0x7f91e0cec960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0cecee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e0cef690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1165 (Thread 0x7f91e0f28640 (LWP 2741167) "apply-low-48"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e0eed288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e0eed288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e0eed288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e0f24798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0eee868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e0eee868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0eedee0, batch=0x7f91e0eed960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e0eedee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e0ef0690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1164 (Thread 0x7f91e1129640 (LWP 2741166) "apply-low-47"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e10ee288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e10ee288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e10ee288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1125798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e10ef868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e10ef868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e10eeee0, batch=0x7f91e10ee960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e10eeee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e10f1690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1163 (Thread 0x7f91e132a640 (LWP 2741165) "apply-low-46"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e12ef288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e12ef288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e12ef288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1326798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e12f0868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e12f0868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e12efee0, batch=0x7f91e12ef960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e12efee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e12f2690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1162 (Thread 0x7f91e152b640 (LWP 2741164) "apply-low-45"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e14f0288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e14f0288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e14f0288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1527798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e14f1868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e14f1868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e14f0ee0, batch=0x7f91e14f0960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e14f0ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e14f3690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1161 (Thread 0x7f91e172c640 (LWP 2741163) "apply-low-44"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e16f1288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e16f1288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e16f1288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1728798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e16f2868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e16f2868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e16f1ee0, batch=0x7f91e16f1960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e16f1ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e16f4690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1160 (Thread 0x7f91e192d640 (LWP 2741162) "apply-low-43"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e18f2288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e18f2288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e18f2288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1929798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e18f3868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e18f3868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e18f2ee0, batch=0x7f91e18f2960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e18f2ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e18f5690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1159 (Thread 0x7f91e1b2e640 (LWP 2741161) "apply-low-42"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e1af3288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e1af3288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e1af3288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1b2a798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1af4868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1af4868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1af3ee0, batch=0x7f91e1af3960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1af3ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e1af6690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1158 (Thread 0x7f91e1d2f640 (LWP 2741160) "apply-low-41"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e1cf4288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e1cf4288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e1cf4288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1d2b798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1cf5868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1cf5868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1cf4ee0, batch=0x7f91e1cf4960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1cf4ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e1cf7690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1157 (Thread 0x7f91e1f30640 (LWP 2741159) "apply-low-40"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e1ef5288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e1ef5288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e1ef5288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e1f2c798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1ef6868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e1ef6868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1ef5ee0, batch=0x7f91e1ef5960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e1ef5ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e1ef8690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1156 (Thread 0x7f91e2131640 (LWP 2741158) "apply-low-39"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e20f6288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e20f6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e20f6288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e212d798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e20f7868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e20f7868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e20f6ee0, batch=0x7f91e20f6960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e20f6ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e20f9690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1155 (Thread 0x7f91e2332640 (LWP 2741157) "apply-low-38"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e22f7288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e22f7288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e22f7288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e232e798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e22f8868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e22f8868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e22f7ee0, batch=0x7f91e22f7960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e22f7ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e22fa690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1154 (Thread 0x7f91e2533640 (LWP 2741156) "apply-low-37"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e24f8288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e24f8288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e24f8288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e252f798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e24f9868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e24f9868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e24f8ee0, batch=0x7f91e24f8960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e24f8ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e24fb690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1153 (Thread 0x7f91e2734640 (LWP 2741155) "apply-low-36"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e26f9288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e26f9288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e26f9288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e2730798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e26fa868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e26fa868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e26f9ee0, batch=0x7f91e26f9960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e26f9ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e26fc690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1152 (Thread 0x7f91e2935640 (LWP 2741154) "apply-low-35"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e28fa288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e28fa288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e28fa288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e2931798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e28fb868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e28fb868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e28faee0, batch=0x7f91e28fa960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e28faee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e28fd690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1151 (Thread 0x7f91e2b36640 (LWP 2741153) "apply-low-34"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e2afb288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e2afb288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e2afb288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e2b32798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2afc868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2afc868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2afbee0, batch=0x7f91e2afb960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2afbee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e2afe690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1150 (Thread 0x7f91e2d37640 (LWP 2741152) "apply-low-33"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e2cfc288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e2cfc288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e2cfc288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e2d33798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2cfd868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2cfd868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2cfcee0, batch=0x7f91e2cfc960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2cfcee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e2cff690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1149 (Thread 0x7f91e2f38640 (LWP 2741151) "apply-low-32"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e2efd288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e2efd288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e2efd288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e2f34798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2efe868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e2efe868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2efdee0, batch=0x7f91e2efd960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e2efdee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e2f00690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1148 (Thread 0x7f91e3139640 (LWP 2741150) "apply-low-31"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e30fe288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e30fe288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e30fe288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3135798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e30ff868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e30ff868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e30feee0, batch=0x7f91e30fe960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e30feee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3101690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1147 (Thread 0x7f91e333a640 (LWP 2741149) "apply-low-30"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e32ff288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e32ff288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e32ff288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3336798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3300868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3300868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e32ffee0, batch=0x7f91e32ff960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e32ffee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3302690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1146 (Thread 0x7f91e353b640 (LWP 2741148) "apply-low-29"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e3500288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e3500288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e3500288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3537798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3501868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3501868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3500ee0, batch=0x7f91e3500960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3500ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3503690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1145 (Thread 0x7f91e39bc640 (LWP 2741147) "apply-low-28"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e3981288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e3981288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e3981288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e39b8798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3982868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3982868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3981ee0, batch=0x7f91e3981960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3981ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3984690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1144 (Thread 0x7f91e3bbd640 (LWP 2741146) "apply-low-27"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e3b82288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e3b82288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e3b82288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3bb9798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3b83868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3b83868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3b82ee0, batch=0x7f91e3b82960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3b82ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3b85690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1143 (Thread 0x7f91e3dbe640 (LWP 2741145) "apply-low-26"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e3d83288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e3d83288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e3d83288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3dba798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3d84868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3d84868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3d83ee0, batch=0x7f91e3d83960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3d83ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3d86690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1142 (Thread 0x7f91e3fbf640 (LWP 2741144) "apply-low-25"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e3f84288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e3f84288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e3f84288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e3fbb798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3f85868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e3f85868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3f84ee0, batch=0x7f91e3f84960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e3f84ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e3f87690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1141 (Thread 0x7f91e41c0640 (LWP 2741143) "apply-low-24"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4185288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4185288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4185288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e41bc798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4186868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4186868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4185ee0, batch=0x7f91e4185960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4185ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e4188690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1140 (Thread 0x7f91e43c1640 (LWP 2741142) "apply-low-23"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4386288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4386288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4386288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e43bd798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4387868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4387868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4386ee0, batch=0x7f91e4386960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4386ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e4389690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1139 (Thread 0x7f91e45c2640 (LWP 2741141) "apply-low-22"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4587288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4587288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4587288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e45be798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4588868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4588868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4587ee0, batch=0x7f91e4587960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4587ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e458a690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1138 (Thread 0x7f91e47c3640 (LWP 2741140) "apply-low-21"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4788288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4788288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4788288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e47bf798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4789868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4789868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4788ee0, batch=0x7f91e4788960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4788ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e478b690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1137 (Thread 0x7f91e49c4640 (LWP 2741139) "apply-low-20"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4989288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4989288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4989288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e49c0798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e498a868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e498a868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4989ee0, batch=0x7f91e4989960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4989ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e498c690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1136 (Thread 0x7f91e4bc5640 (LWP 2741138) "apply-low-19"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4b8a288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4b8a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4b8a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e4bc1798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4b8b868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4b8b868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4b8aee0, batch=0x7f91e4b8a960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4b8aee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e4b8d690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1135 (Thread 0x7f91e4dc6640 (LWP 2741137) "apply-low-18"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4d8b288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4d8b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4d8b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e4dc2798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4d8c868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4d8c868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4d8bee0, batch=0x7f91e4d8b960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4d8bee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e4d8e690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1134 (Thread 0x7f91e4fc7640 (LWP 2741136) "apply-low-17"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e4f8c288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e4f8c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e4f8c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e4fc3798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4f8d868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e4f8d868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4f8cee0, batch=0x7f91e4f8c960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e4f8cee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e4f8f690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1133 (Thread 0x7f91e5448640 (LWP 2741135) "apply-low-16"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e540d288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e540d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e540d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5444798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e540e868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e540e868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e540dee0, batch=0x7f91e540d960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e540dee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5410690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1132 (Thread 0x7f91e5649640 (LWP 2741134) "apply-low-15"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e560e288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e560e288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e560e288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5645798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e560f868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e560f868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e560eee0, batch=0x7f91e560e960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e560eee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5611690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1131 (Thread 0x7f91e584a640 (LWP 2741133) "apply-low-14"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e580f288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e580f288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e580f288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5846798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5810868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5810868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e580fee0, batch=0x7f91e580f960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e580fee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5812690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1130 (Thread 0x7f91e5a4b640 (LWP 2741132) "apply-low-13"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e5a10288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e5a10288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e5a10288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5a47798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5a11868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5a11868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5a10ee0, batch=0x7f91e5a10960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5a10ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5a13690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1129 (Thread 0x7f91e5c4c640 (LWP 2741131) "apply-low-12"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e5c11288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e5c11288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e5c11288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5c48798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5c12868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5c12868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5c11ee0, batch=0x7f91e5c11960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5c11ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5c14690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1128 (Thread 0x7f91e5e4d640 (LWP 2741130) "apply-low-11"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e5e12288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e5e12288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e5e12288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e5e49798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5e13868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e5e13868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5e12ee0, batch=0x7f91e5e12960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e5e12ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e5e15690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1127 (Thread 0x7f91e604e640 (LWP 2741129) "apply-low-10"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6013288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6013288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6013288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e604a798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6014868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6014868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6013ee0, batch=0x7f91e6013960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6013ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6016690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1126 (Thread 0x7f91e624f640 (LWP 2741128) "apply-low-9"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6214288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6214288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6214288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e624b798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6215868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6215868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6214ee0, batch=0x7f91e6214960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6214ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6217690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1125 (Thread 0x7f91e6450640 (LWP 2741127) "apply-low-8"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6415288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6415288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6415288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e644c798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6416868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6416868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6415ee0, batch=0x7f91e6415960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6415ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6418690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1124 (Thread 0x7f91e6651640 (LWP 2741126) "apply-low-7"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6616288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6616288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6616288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e664d798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6617868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6617868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6616ee0, batch=0x7f91e6616960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6616ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6619690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1123 (Thread 0x7f91e6852640 (LWP 2741125) "apply-low-6"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6817288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6817288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6817288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e684e798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6818868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6818868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6817ee0, batch=0x7f91e6817960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6817ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e681a690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1122 (Thread 0x7f91e6a53640 (LWP 2741124) "apply-low-5"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6a18288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6a18288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6a18288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e6a4f798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6a19868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6a19868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6a18ee0, batch=0x7f91e6a18960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6a18ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6a1b690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1121 (Thread 0x7f91e6c54640 (LWP 2741123) "apply-low-4"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6c19288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6c19288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6c19288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e6c50798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6c1a868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6c1a868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6c19ee0, batch=0x7f91e6c19960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6c19ee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6c1c690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1120 (Thread 0x7f91e6e55640 (LWP 2741122) "apply-low-3"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e6e1a288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e6e1a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e6e1a288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e6e51798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6e1b868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e6e1b868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6e1aee0, batch=0x7f91e6e1a960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e6e1aee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e6e1d690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1119 (Thread 0x7f91e7056640 (LWP 2741121) "apply-low-2"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e701b288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e701b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e701b288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e7052798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e701c868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e701c868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e701bee0, batch=0x7f91e701b960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e701bee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e701e690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1118 (Thread 0x7f91e7257640 (LWP 2741120) "apply-low-1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e721c288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e721c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e721c288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e7253798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e721d868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e721d868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e721cee0, batch=0x7f91e721c960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e721cee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e721f690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1117 (Thread 0x7f91e7458640 (LWP 2741119) "apply-low-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e741d288, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d6581a7 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::apply::ControlFsm>> (cx=0x7f91e741d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d647042 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cx=0x7f91e741d288) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d645d33 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::apply::ControlFsm>>, ()> (cell=0x7f91e7454798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86de1b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d64060e in crossbeam_channel::context::Context::with, raftstore::store::fsm::apply::ControlFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d65788f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f956a8cfc00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3d3de in crossbeam_channel::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e741e868) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb363c2 in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e741e868) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e741dee0, batch=0x7f91e741d960) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e741dee0) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e7420690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1116 (Thread 0x7f91e7659640 (LWP 2741118) "apply-3"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f957cb32c27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91e7655750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f957ca0db44 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91e7655750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f957a7d0e86 in parking_lot_core::parking_lot::park::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (thread_data=0x7f91e7655730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f957a7d0105 in parking_lot_core::parking_lot::with_thread_data, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (key=140279713980288, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f957b690e57 in tikv_util::mpsc::priority_queue::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e761f870) at components/tikv_util/src/mpsc/priority_queue.rs:175 -#7 0x00007f957cb363dd in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e761f868) at components/resource_control/src/channel.rs:158 -#8 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e761eee0, batch=0x7f91e761e960) at components/batch-system/src/batch.rs:373 -#9 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e761eee0) at components/batch-system/src/batch.rs:393 -#10 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#11 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#12 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#13 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#14 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#15 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e7621690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#16 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#17 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#18 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#19 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#20 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#21 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#22 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#23 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#24 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#25 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1115 (Thread 0x7f91e785a640 (LWP 2741117) "apply-2"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f957cb32c27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91e7856750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f957ca0db44 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91e7856750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f957a7d0e86 in parking_lot_core::parking_lot::park::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (thread_data=0x7f91e7856730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f957a7d0105 in parking_lot_core::parking_lot::with_thread_data, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (key=140279713980288, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f957b690e57 in tikv_util::mpsc::priority_queue::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7820870) at components/tikv_util/src/mpsc/priority_queue.rs:175 -#7 0x00007f957cb363dd in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7820868) at components/resource_control/src/channel.rs:158 -#8 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e781fee0, batch=0x7f91e781f960) at components/batch-system/src/batch.rs:373 -#9 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e781fee0) at components/batch-system/src/batch.rs:393 -#10 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#11 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#12 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#13 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#14 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#15 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e7822690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#16 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#17 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#18 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#19 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#20 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#21 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#22 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#23 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#24 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#25 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1114 (Thread 0x7f91e7a5b640 (LWP 2741116) "apply-1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f957cb32c27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91e7a57750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f957ca0db44 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91e7a57750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f957a7d0e86 in parking_lot_core::parking_lot::park::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (thread_data=0x7f91e7a57730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f957a7d0105 in parking_lot_core::parking_lot::with_thread_data, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (key=140279713980288, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f957b690e57 in tikv_util::mpsc::priority_queue::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7a21870) at components/tikv_util/src/mpsc/priority_queue.rs:175 -#7 0x00007f957cb363dd in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7a21868) at components/resource_control/src/channel.rs:158 -#8 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e7a20ee0, batch=0x7f91e7a20960) at components/batch-system/src/batch.rs:373 -#9 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e7a20ee0) at components/batch-system/src/batch.rs:393 -#10 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#11 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#12 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#13 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#14 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#15 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e7a23690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#16 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#17 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#18 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#19 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#20 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#21 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#22 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#23 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#24 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#25 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1113 (Thread 0x7f91e7c5c640 (LWP 2741115) "apply-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f957cb32c27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91e7c58750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f957ca0db44 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91e7c58750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f957a7d0e86 in parking_lot_core::parking_lot::park::{closure#0}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (thread_data=0x7f91e7c58730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f957a7d0105 in parking_lot_core::parking_lot::with_thread_data, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}, raftstore::store::fsm::apply::ControlFsm>>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}, raftstore::store::fsm::apply::ControlFsm>>> (key=140279713980288, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f957b690e57 in tikv_util::mpsc::priority_queue::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7c22870) at components/tikv_util/src/mpsc/priority_queue.rs:175 -#7 0x00007f957cb363dd in resource_control::channel::Receiver, raftstore::store::fsm::apply::ControlFsm>>::recv, raftstore::store::fsm::apply::ControlFsm>> (self=0x7f91e7c22868) at components/resource_control/src/channel.rs:158 -#8 0x00007f957b9a9833 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::fetch_fsm, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e7c21ee0, batch=0x7f91e7c21960) at components/batch-system/src/batch.rs:373 -#9 0x00007f957b9a4f09 in batch_system::batch::Poller, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller>::poll, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::ApplyPoller> (self=0x7f91e7c21ee0) at components/batch-system/src/batch.rs:393 -#10 0x00007f957b99aad0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder> () at components/batch-system/src/batch.rs:588 -#11 0x00007f957df41ba7 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#12 0x00007f957a80ae47 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#13 0x00007f957cb7a824 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#14 0x00007f957c9e6214 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#15 0x00007f957be12b70 in std::panicking::try::do_call, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (data=0x7f91e7c24690) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#16 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#17 0x00007f957bdb0377 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#18 0x00007f957cb79e33 in std::panic::catch_unwind, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#19 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#20 0x00007f957d67e72f in core::ops::function::FnOnce::call_once, raftstore::store::fsm::apply::ControlFsm, raftstore::store::fsm::apply::Builder>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#21 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#22 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#23 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#24 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#25 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1112 (Thread 0x7f91e7e5d640 (LWP 2741114) "time updater"): -#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 -#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 -#2 0x00007f9583cea820 in std::sys::unix::thread::Thread::sleep () at library/std/src/sys/unix/thread.rs:243 -#3 std::thread::sleep () at library/std/src/thread/mod.rs:870 -#4 0x00007f9582bdbe50 in prometheus::timer::ensure_updater::{closure#0} () at src/timer.rs:57 -#5 0x00007f9582bf32e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#6 0x00007f9582bf5a27 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#7 0x00007f9582c14c67 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#8 0x00007f9582bb5d0d in std::panicking::try::do_call>, ()> (data=0x7f91e7e29af0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#9 0x00007f9582be891b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#10 0x00007f9582bb5ca6 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#11 0x00007f9582bf5879 in std::panic::catch_unwind>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#12 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#13 0x00007f9582bb62af in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#14 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#15 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#16 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#17 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#18 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1111 (Thread 0x7f91e805e640 (LWP 2741113) "raftstore-135-3"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e8006378, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d658057 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::store::StoreFsm>> (cx=0x7f91e8006378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d645832 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::store::StoreFsm>>, ()> (cx=0x7f91e8006378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d647163 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::store::StoreFsm>>, ()> (cell=0x7f91e805a798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86375b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::store::StoreFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d641aee in crossbeam_channel::context::Context::with, raftstore::store::fsm::store::StoreFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d657b8f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f956a8cfe00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3cfde in crossbeam_channel::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e800a378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb36512 in resource_control::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e800a378) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a96d3 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::fetch_fsm, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8006fd8, batch=0x7f91e8006a50) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a65d9 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::poll, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8006fd8) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99a9b0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df423f6 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae87 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a66d in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e3441 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957bdf0311 in std::panicking::try::do_call, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (data=0x7f91e8013ff0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bd874f2 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb781ce in std::panic::catch_unwind, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e3cf in core::ops::function::FnOnce::call_once, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1110 (Thread 0x7f91e825f640 (LWP 2741112) "raftstore-135-2"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e8207378, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d658057 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::store::StoreFsm>> (cx=0x7f91e8207378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d645832 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::store::StoreFsm>>, ()> (cx=0x7f91e8207378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d647163 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::store::StoreFsm>>, ()> (cell=0x7f91e825b798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86375b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::store::StoreFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d641aee in crossbeam_channel::context::Context::with, raftstore::store::fsm::store::StoreFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d657b8f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f956a8cfe00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3cfde in crossbeam_channel::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e820b378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb36512 in resource_control::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e820b378) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a96d3 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::fetch_fsm, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8207fd8, batch=0x7f91e8207a50) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a65d9 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::poll, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8207fd8) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99a9b0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df423f6 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae87 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a66d in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e3441 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957bdf0311 in std::panicking::try::do_call, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (data=0x7f91e8214ff0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bd874f2 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb781ce in std::panic::catch_unwind, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e3cf in core::ops::function::FnOnce::call_once, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1109 (Thread 0x7f91e8460640 (LWP 2741111) "raftstore-135-1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e8408378, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d658057 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::store::StoreFsm>> (cx=0x7f91e8408378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d645832 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::store::StoreFsm>>, ()> (cx=0x7f91e8408378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d647163 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::store::StoreFsm>>, ()> (cell=0x7f91e845c798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86375b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::store::StoreFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d641aee in crossbeam_channel::context::Context::with, raftstore::store::fsm::store::StoreFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d657b8f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f956a8cfe00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3cfde in crossbeam_channel::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e840c378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb36512 in resource_control::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e840c378) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a96d3 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::fetch_fsm, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8408fd8, batch=0x7f91e8408a50) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a65d9 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::poll, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8408fd8) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99a9b0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df423f6 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae87 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a66d in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e3441 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957bdf0311 in std::panicking::try::do_call, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (data=0x7f91e8415ff0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bd874f2 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb781ce in std::panic::catch_unwind, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e3cf in core::ops::function::FnOnce::call_once, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1108 (Thread 0x7f91e8661640 (LWP 2741110) "raftstore-135-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f957cb3f2ab in crossbeam_channel::context::Context::wait_until (self=0x7f91e8609378, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f957d658057 in crossbeam_channel::flavors::list::{impl#3}::recv::{closure#1}, raftstore::store::fsm::store::StoreFsm>> (cx=0x7f91e8609378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:495 -#6 0x00007f957d645832 in crossbeam_channel::context::{impl#0}::with::{closure#0}, raftstore::store::fsm::store::StoreFsm>>, ()> (cx=0x7f91e8609378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f957d647163 in crossbeam_channel::context::{impl#0}::with::{closure#1}, raftstore::store::fsm::store::StoreFsm>>, ()> (cell=0x7f91e865d798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f957a86375b in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, raftstore::store::fsm::store::StoreFsm>>, ()>, ()> (self=0x7f9583d7cb40, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957d641aee in crossbeam_channel::context::Context::with, raftstore::store::fsm::store::StoreFsm>>, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f957d657b8f in crossbeam_channel::flavors::list::Channel, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f956a8cfe00, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/list.rs:485 -#11 0x00007f957cb3cfde in crossbeam_channel::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e860d378) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:815 -#12 0x00007f957cb36512 in resource_control::channel::Receiver, raftstore::store::fsm::store::StoreFsm>>::recv, raftstore::store::fsm::store::StoreFsm>> (self=0x7f91e860d378) at components/resource_control/src/channel.rs:157 -#13 0x00007f957b9a96d3 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::fetch_fsm, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8609fd8, batch=0x7f91e8609a50) at components/batch-system/src/batch.rs:373 -#14 0x00007f957b9a65d9 in batch_system::batch::Poller, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>::poll, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPoller, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> (self=0x7f91e8609fd8) at components/batch-system/src/batch.rs:393 -#15 0x00007f957b99a9b0 in batch_system::batch::{impl#9}::start_poller::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>> () at components/batch-system/src/batch.rs:588 -#16 0x00007f957df423f6 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#17 0x00007f957a80ae87 in std::sys_common::backtrace::__rust_begin_short_backtrace, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#18 0x00007f957cb7a66d in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#19 0x00007f957c9e3441 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#20 0x00007f957bdf0311 in std::panicking::try::do_call, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (data=0x7f91e8616ff0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#21 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#22 0x00007f957bd874f2 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#23 0x00007f957cb781ce in std::panic::catch_unwind, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#24 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#25 0x00007f957d67e3cf in core::ops::function::FnOnce::call_once, raftstore::store::fsm::store::StoreFsm, raftstore::store::fsm::store::RaftPollerBuilder, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#26 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#27 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1107 (Thread 0x7f91e8862640 (LWP 2741109) "steady-timer"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cfc2f4 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 0x00007f9583d0039a in std::sys::unix::locks::futex_condvar::Condvar::wait_optional_timeout () at library/std/src/sys/unix/locks/futex_condvar.rs:49 -#3 std::sys::unix::locks::futex_condvar::Condvar::wait_timeout () at library/std/src/sys/unix/locks/futex_condvar.rs:37 -#4 0x00007f958327577d in std::sync::condvar::Condvar::wait_timeout<()> (self=0x7f94aeac63e0, guard=..., dur=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/condvar.rs:385 -#5 0x00007f958327db79 in crossbeam_utils::sync::parker::Inner::park (self=0x7f94aeac63d0, deadline=...) at src/sync/parker.rs:367 -#6 0x00007f958327d5b2 in crossbeam_utils::sync::parker::Parker::park_deadline (self=0x7f91e885e5c0, deadline=...) at src/sync/parker.rs:145 -#7 0x00007f958327d567 in crossbeam_utils::sync::parker::Parker::park_timeout (self=0x7f91e885e5c0, timeout=...) at src/sync/parker.rs:125 -#8 0x00007f9582a0a02a in tokio_executor::park::{impl#3}::park_timeout::{closure#0} (inner=0x7f91e885e5c0) at src/park.rs:215 -#9 0x00007f9582a09eeb in tokio_executor::park::{impl#2}::with_current::{closure#0} (inner=0x7f91e885e5c0) at src/park.rs:196 -#10 0x00007f9582a0a347 in std::thread::local::LocalKey::try_with, ()> (self=0x7f958406cfc8, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#11 0x00007f9582a0a214 in std::thread::local::LocalKey::with, ()> (self=0x7f958406cfc8, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#12 0x00007f9582a09ec3 in tokio_executor::park::ParkThread::with_current (self=0x7f91e882e810, f=...) at src/park.rs:196 -#13 0x00007f9582a09fe3 in tokio_executor::park::{impl#3}::park_timeout (self=0x7f91e882e810, duration=...) at src/park.rs:215 -#14 0x00007f95828c9a5c in tokio_timer::timer::{impl#4}::park (self=0x7f91e882e7d8) at /home/xzx/.cargo/git/checkouts/tokio-8e927faba632ed16/4394380/tokio-timer/src/timer/mod.rs:369 -#15 0x00007f95828a52c9 in tokio_timer::timer::Timer::turn (self=0x7f91e882e7d8, max_wait=...) at /home/xzx/.cargo/git/checkouts/tokio-8e927faba632ed16/4394380/tokio-timer/src/timer/mod.rs:256 -#16 0x00007f958289e7f9 in tikv_util::timer::start_timer_thread::{closure#0} () at components/tikv_util/src/timer.rs:259 -#17 0x00007f95828de219 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, ()> () at components/tikv_util/src/sys/thread.rs:441 -#18 0x00007f95828ad357 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#19 0x00007f95828b81ad in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#20 0x00007f958285eec1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#21 0x00007f958286a625 in std::panicking::try::do_call, ()>, ()>>, ()> (data=0x7f91e882e9e0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#22 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#23 0x00007f9582869e7f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#24 0x00007f95828b72e1 in std::panic::catch_unwind, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#25 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#26 0x00007f95827ed88f in core::ops::function::FnOnce::call_once, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#27 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#28 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#29 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#30 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#31 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1106 (Thread 0x7f91e8a63640 (LWP 2741052) "store-writer-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f957cb32c27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91e8a5f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f957ca0db44 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91e8a5f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f957a7d2746 in parking_lot_core::parking_lot::park::{closure#0}>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}>> (thread_data=0x7f91e8a5f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f957a7cff75 in parking_lot_core::parking_lot::with_thread_data>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#1}>, tikv_util::mpsc::priority_queue::{impl#8}::recv::{closure_env#2}>> (key=140266244010880, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f957b690f6a in tikv_util::mpsc::priority_queue::Receiver>::recv> (self=0x7f91e8a27660) at components/tikv_util/src/mpsc/priority_queue.rs:175 -#7 0x00007f957cb3644d in resource_control::channel::Receiver>::recv> (self=0x7f91e8a27658) at components/resource_control/src/channel.rs:158 -#8 0x00007f957b682b5f in raftstore::store::async_io::write::Worker, tikv::server::transport::ServerTransport, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>>::run, tikv::server::transport::ServerTransport, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>> (self=0x7f91e8a26ee8) at components/raftstore/src/store/async_io/write.rs:762 -#9 0x00007f957b671ade in raftstore::store::async_io::write::{impl#10}::increase_to::{closure#0}::{closure#0}, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter> () at components/raftstore/src/store/async_io/write.rs:1193 -#10 0x00007f957df42539 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#11 0x00007f957a80adf7 in std::sys_common::backtrace::__rust_begin_short_backtrace, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#12 0x00007f957cb7a6a0 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#13 0x00007f957c9da2a4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#14 0x00007f957be16364 in std::panicking::try::do_call, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()>>, ()> (data=0x7f91e8a2a240) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#15 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#16 0x00007f957bdb9517 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#17 0x00007f957cb779a9 in std::panic::catch_unwind, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#18 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#19 0x00007f957d67f34f in core::ops::function::FnOnce::call_once, engine_tiflash::engine::details::RocksEngine>, tikv::server::resolve::PdStoreAddrResolver>, raftstore::store::fsm::store::RaftRouter>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#20 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#22 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#23 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#24 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1105 (Thread 0x7f91eec64640 (LWP 2741048) "refreash-config"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91eec60750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91eec60750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91eec60730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248948432, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91eec2fc70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91eec2fb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91eec30170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1104 (Thread 0x7f91eee65640 (LWP 2741047) "raftlog-fetch-w"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91eee61750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91eee61750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91eee61730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248947792, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91eee30c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91eee30b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91eee31170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1103 (Thread 0x7f91ef066640 (LWP 2741046) "region-worker-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ef062750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ef062750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ef062730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248947152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ef031c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91ef031b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91ef032170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1102 (Thread 0x7f91ef4e7640 (LWP 2741045) "cleanup-worker-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ef4e3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ef4e3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ef4e3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248946512, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ef4b2c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91ef4b2b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91ef4b3170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1101 (Thread 0x7f91ef6e8640 (LWP 2741044) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ef6e4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ef6e4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ef6e4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ef6b3c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91ef6b3b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91ef6b4170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1100 (Thread 0x7f91ef8e9640 (LWP 2741043) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ef8e5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ef8e5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ef8e5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ef8b4c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91ef8b4b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91ef8b5170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1099 (Thread 0x7f91efaea640 (LWP 2741042) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91efae6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91efae6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91efae6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91efab5c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91efab5b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91efab6170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1098 (Thread 0x7f91efceb640 (LWP 2741041) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91efce7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91efce7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91efce7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91efcb6c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91efcb6b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91efcb7170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1097 (Thread 0x7f91efeec640 (LWP 2741040) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91efee8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91efee8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91efee8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91efeb7c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91efeb7b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91efeb8170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1096 (Thread 0x7f91f00ed640 (LWP 2741039) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f00e9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f00e9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f00e9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f00b8c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f00b8b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f00b9170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1095 (Thread 0x7f91f02ee640 (LWP 2741038) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f02ea750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f02ea750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f02ea730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f02b9c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f02b9b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f02ba170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1094 (Thread 0x7f91f04ef640 (LWP 2741037) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f04eb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f04eb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f04eb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f04bac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f04bab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f04bb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1093 (Thread 0x7f91f06f0640 (LWP 2741036) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f06ec750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f06ec750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f06ec730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f06bbc70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f06bbb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f06bc170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1092 (Thread 0x7f91f08f1640 (LWP 2741035) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f08ed750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f08ed750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f08ed730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f08bcc70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f08bcb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f08bd170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1091 (Thread 0x7f91f0af2640 (LWP 2741034) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f0aee750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f0aee750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f0aee730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f0abdc70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f0abdb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f0abe170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1090 (Thread 0x7f91f0cf3640 (LWP 2741033) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f0cef750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f0cef750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f0cef730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f0cbec70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f0cbeb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f0cbf170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1089 (Thread 0x7f91f0ef4640 (LWP 2741032) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f0ef0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f0ef0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f0ef0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f0ebfc70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f0ebfb70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f0ec0170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1088 (Thread 0x7f91f10f5640 (LWP 2741031) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f10f1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f10f1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f10f1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f10c0c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f10c0b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f10c1170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1087 (Thread 0x7f91f12f6640 (LWP 2741030) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f12f2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f12f2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f12f2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f12c1c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f12c1b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f12c2170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1086 (Thread 0x7f91f14f7640 (LWP 2741029) "snap-generator-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f14f3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f14f3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f14f3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945872, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f14c2c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f14c2b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f14c3170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1085 (Thread 0x7f91f16f8640 (LWP 2741028) "purge-worker-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f16f4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f16f4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f16f4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248945232, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f16c3c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f16c3b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f16c4170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1084 (Thread 0x7f91f18f9640 (LWP 2741027) "region-task-15"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f18f5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f18f5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f18f5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f18c5648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f18c5648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f18c5860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1083 (Thread 0x7f91f1afa640 (LWP 2741026) "region-task-14"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f1af6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f1af6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f1af6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f1ac6648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f1ac6648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f1ac6860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1082 (Thread 0x7f91f1cfb640 (LWP 2741025) "region-task-13"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f1cf7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f1cf7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f1cf7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f1cc7648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f1cc7648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f1cc7860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1081 (Thread 0x7f91f1efc640 (LWP 2741024) "region-task-12"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f1ef8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f1ef8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f1ef8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f1ec8648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f1ec8648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f1ec8860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1080 (Thread 0x7f91f20fd640 (LWP 2741023) "region-task-11"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f20f9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f20f9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f20f9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f20c9648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f20c9648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f20c9860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1079 (Thread 0x7f91f22fe640 (LWP 2741022) "region-task-10"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f22fa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f22fa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f22fa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f22ca648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f22ca648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f22ca860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1078 (Thread 0x7f91f24ff640 (LWP 2741021) "region-task-9"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f24fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f24fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f24fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f24cb648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f24cb648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f24cb860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1077 (Thread 0x7f91f2700640 (LWP 2741020) "region-task-8"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f26fc750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f26fc750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f26fc730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f26cc648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f26cc648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f26cc860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1076 (Thread 0x7f91f2901640 (LWP 2741019) "region-task-7"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f28fd750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f28fd750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f28fd730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f28cd648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f28cd648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f28cd860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1075 (Thread 0x7f91f2b02640 (LWP 2741018) "region-task-6"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f2afe750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f2afe750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f2afe730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f2ace648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f2ace648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f2ace860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1074 (Thread 0x7f91f2d03640 (LWP 2741017) "region-task-5"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f2cff750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f2cff750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f2cff730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f2ccf648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f2ccf648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f2ccf860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1073 (Thread 0x7f91f2f04640 (LWP 2741016) "region-task-4"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f2f00750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f2f00750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f2f00730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f2ed0648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f2ed0648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f2ed0860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1072 (Thread 0x7f91f3105640 (LWP 2741015) "region-task-3"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3101750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3101750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3101730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f30d1648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f30d1648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f30d1860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1071 (Thread 0x7f91f3306640 (LWP 2741014) "region-task-2"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3302750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3302750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3302730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f32d2648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f32d2648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f32d2860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1070 (Thread 0x7f91f3507640 (LWP 2741013) "region-task-1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3503750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3503750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3503730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f34d3648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f34d3648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f34d3860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1069 (Thread 0x7f91f3708640 (LWP 2741012) "region-task-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3704750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3704750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3704730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944592, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f36d4648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f36d4648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f36d4860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1068 (Thread 0x7f91f3909640 (LWP 2741011) "raft-stream-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3905750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3905750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3905730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248944432, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f38d5648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f91f38d5648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f91f38d5860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1067 (Thread 0x7f91f3b0a640 (LWP 2741004) "snap-handler-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3b06750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3b06750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f3b06730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266248943792, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f3ad5c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f91f3ad5b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f91f3ad6170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1066 (Thread 0x7f91f3d0b640 (LWP 2741003) "transport-stats"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f3d07750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f3d07750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f91f3d07730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140279560138632, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f956159ef88, mutex=0x7f956159ef90, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f956159ef88, mutex_guard=0x7f91f3cd6368) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f956159ef88, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830ef2f6 in tokio::runtime::park::Inner::park (self=0x7f956159ef80) at src/runtime/park.rs:116 -#10 0x00007f95830ef113 in tokio::runtime::park::ParkThread::park (self=0x7f95615941a0) at src/runtime/park.rs:62 -#11 0x00007f95830f1adf in tokio::runtime::driver::IoStack::park (self=0x7f9561594198, handle=0x7f956a8ced18) at src/runtime/driver.rs:176 -#12 0x00007f95830f2090 in tokio::runtime::driver::TimeDriver::park (self=0x7f9561594190, handle=0x7f956a8ced18) at src/runtime/driver.rs:333 -#13 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f9561594190, handle=0x7f956a8ced18) at src/runtime/driver.rs:71 -#14 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f9247f23010, driver=0x7f9561594190, handle=0x7f956a8ced18) at src/runtime/scheduler/multi_thread/park.rs:184 -#15 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f9247f23010, handle=0x7f956a8ced18) at src/runtime/scheduler/multi_thread/park.rs:117 -#16 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91f3cd69a0, handle=0x7f956a8ced18) at src/runtime/scheduler/multi_thread/park.rs:67 -#17 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91f3cd6e98, core=0x7f956159b920, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#18 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91f3cd6e98, core=0x7f956159b920) at src/runtime/scheduler/multi_thread/worker.rs:702 -#19 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91f3cd6e98, core=0x7f956159b920) at src/runtime/scheduler/multi_thread/worker.rs:553 -#20 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#21 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91f3cd8610, t=0x7f91f3cd6e90, f=...) at src/runtime/context/scoped.rs:40 -#22 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91f3cd85d8) at src/runtime/context.rs:180 -#23 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#24 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#25 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91f3cd6e90, f=...) at src/runtime/context.rs:180 -#26 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#27 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91f3cd7098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#28 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#29 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#30 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91f3cd7220) at src/runtime/blocking/task.rs:42 -#31 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594228) at src/runtime/task/core.rs:328 -#32 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594228, f=...) at src/loom/std/unsafe_cell.rs:16 -#33 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594220, cx=...) at src/runtime/task/core.rs:317 -#34 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#35 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#36 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91f3cd7398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#37 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#38 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#39 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#40 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594220, cx=...) at src/runtime/task/harness.rs:473 -#41 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91f3cd75b0) at src/runtime/task/harness.rs:208 -#42 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#43 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#44 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#45 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#46 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#47 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f95615afc50, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 -#48 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#49 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#50 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#51 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#52 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91f3cd7a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#53 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#54 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#55 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#56 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#57 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#58 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#59 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#60 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#61 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#62 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1065 (Thread 0x7f91f3f0c640 (LWP 2741001) "sst_import_misc"): -#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 -#1 0x00007f9583185778 in mio::sys::unix::selector::epoll::Selector::select (self=0x7f9561507f38, events=0x7f9561507f18, timeout=...) at src/sys/unix/selector/epoll.rs:97 -#2 0x00007f9583184fd1 in mio::poll::Poll::poll (self=0x7f9561507f30, events=0x7f9561507f18, timeout=...) at src/poll.rs:426 -#3 0x00007f9583167b7c in tokio::runtime::io::driver::Driver::turn (self=0x7f9561507f18, handle=0x7f956a8ceb18, max_wait=...) at src/runtime/io/driver.rs:149 -#4 0x00007f958316786a in tokio::runtime::io::driver::Driver::park_timeout (self=0x7f9561507f18, rt_handle=0x7f956a8ceb18, duration=...) at src/runtime/io/driver.rs:127 -#5 0x00007f958316a8c0 in tokio::runtime::signal::Driver::park_timeout (self=0x7f9561507f18, handle=0x7f956a8ceb18, duration=...) at src/runtime/signal/mod.rs:97 -#6 0x00007f95831527c0 in tokio::runtime::process::Driver::park_timeout (self=0x7f9561507f18, handle=0x7f956a8ceb18, duration=...) at src/runtime/process.rs:37 -#7 0x00007f95830f1b55 in tokio::runtime::driver::IoStack::park_timeout (self=0x7f9561507f18, handle=0x7f956a8ceb18, duration=...) at src/runtime/driver.rs:182 -#8 0x00007f958316a49c in tokio::runtime::time::Driver::park_thread_timeout (self=0x7f9561507f18, rt_handle=0x7f956a8ceb18, duration=...) at src/runtime/time/mod.rs:270 -#9 0x00007f958316a1f5 in tokio::runtime::time::Driver::park_internal (self=0x7f9561507f18, rt_handle=0x7f956a8ceb18, limit=...) at src/runtime/time/mod.rs:226 -#10 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f9561507f18, handle=0x7f956a8ceb18) at src/runtime/time/mod.rs:165 -#11 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f9561507f10, handle=0x7f956a8ceb18) at src/runtime/driver.rs:332 -#12 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f9561507f10, handle=0x7f956a8ceb18) at src/runtime/driver.rs:71 -#13 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f956159ead0, driver=0x7f9561507f10, handle=0x7f956a8ceb18) at src/runtime/scheduler/multi_thread/park.rs:184 -#14 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f956159ead0, handle=0x7f956a8ceb18) at src/runtime/scheduler/multi_thread/park.rs:117 -#15 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f91f3ed79a0, handle=0x7f956a8ceb18) at src/runtime/scheduler/multi_thread/park.rs:67 -#16 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f91f3ed7e98, core=0x7f956159b6a0, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#17 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f91f3ed7e98, core=0x7f956159b6a0) at src/runtime/scheduler/multi_thread/worker.rs:702 -#18 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f91f3ed7e98, core=0x7f956159b6a0) at src/runtime/scheduler/multi_thread/worker.rs:553 -#19 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#20 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f91f3ed9610, t=0x7f91f3ed7e90, f=...) at src/runtime/context/scoped.rs:40 -#21 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f91f3ed95d8) at src/runtime/context.rs:180 -#22 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#23 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#24 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f91f3ed7e90, f=...) at src/runtime/context.rs:180 -#25 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#26 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f91f3ed8098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#27 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#28 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#29 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f91f3ed8220) at src/runtime/blocking/task.rs:42 -#30 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561507fa8) at src/runtime/task/core.rs:328 -#31 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561507fa8, f=...) at src/loom/std/unsafe_cell.rs:16 -#32 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561507fa0, cx=...) at src/runtime/task/core.rs:317 -#33 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#34 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#35 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f91f3ed8398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#36 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#37 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#38 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#39 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561507fa0, cx=...) at src/runtime/task/harness.rs:473 -#40 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f91f3ed85b0) at src/runtime/task/harness.rs:208 -#41 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#42 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#43 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#44 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#45 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#46 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f95615af9d0, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 -#47 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#48 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#49 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#50 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#51 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f91f3ed8a20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#52 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#53 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#54 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#55 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#56 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#57 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#58 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#59 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#60 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#61 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1064 (Thread 0x7f91f410d640 (LWP 2740998) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f4109750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f4109750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f4109730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f40d7be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f40d7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f40d85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1063 (Thread 0x7f91f430e640 (LWP 2740997) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f430a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f430a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f430a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f42d8be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f42d8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f42d95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1062 (Thread 0x7f91f450f640 (LWP 2740996) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f450b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f450b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f450b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f44d9be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f44d9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f44da5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1061 (Thread 0x7f91f4710640 (LWP 2740995) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f470c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f470c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f470c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f46dabe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f46da938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f46db5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1060 (Thread 0x7f91f4911640 (LWP 2740994) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f490d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f490d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f490d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f48dbbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f48db938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f48dc5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1059 (Thread 0x7f91f4b12640 (LWP 2740993) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f4b0e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f4b0e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f4b0e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f4adcbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f4adc938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f4add5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1058 (Thread 0x7f91f4d13640 (LWP 2740992) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f4d0f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f4d0f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f4d0f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f4cddbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f4cdd938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f4cde5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1057 (Thread 0x7f91f4f14640 (LWP 2740991) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f4f10750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f4f10750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f4f10730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f4edebe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f4ede938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f4edf5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1056 (Thread 0x7f91f5115640 (LWP 2740990) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5111750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5111750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5111730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f50dfbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f50df938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f50e05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1055 (Thread 0x7f91f5316640 (LWP 2740989) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5312750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5312750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5312730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f52e0be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f52e0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f52e15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1054 (Thread 0x7f91f5517640 (LWP 2740988) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5513750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5513750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5513730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f54e1be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f54e1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f54e25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1053 (Thread 0x7f91f5718640 (LWP 2740987) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5714750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5714750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5714730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f56e2be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f56e2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f56e35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1052 (Thread 0x7f91f5919640 (LWP 2740986) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5915750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5915750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5915730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f58e3be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f58e3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f58e45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1051 (Thread 0x7f91f5b1a640 (LWP 2740985) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5b16750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5b16750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5b16730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f5ae4be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f5ae4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f5ae55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1050 (Thread 0x7f91f5d1b640 (LWP 2740984) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5d17750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5d17750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5d17730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f5ce5be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f5ce5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f5ce65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1049 (Thread 0x7f91f5f1c640 (LWP 2740983) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f5f18750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f5f18750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f5f18730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f5ee6be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f5ee6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f5ee75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1048 (Thread 0x7f91f611d640 (LWP 2740982) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f6119750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f6119750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f6119730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f60e7be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f60e7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f60e85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1047 (Thread 0x7f91f631e640 (LWP 2740981) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f631a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f631a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f631a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f62e8be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f62e8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f62e95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1046 (Thread 0x7f91f651f640 (LWP 2740980) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f651b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f651b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f651b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f64e9be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f64e9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f64ea5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1045 (Thread 0x7f91f6720640 (LWP 2740979) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f671c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f671c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f671c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f66eabe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f66ea938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f66eb5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1044 (Thread 0x7f91f6921640 (LWP 2740978) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f691d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f691d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f691d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f68ebbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f68eb938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f68ec5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1043 (Thread 0x7f91f6b22640 (LWP 2740977) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f6b1e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f6b1e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f6b1e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f6aecbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f6aec938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f6aed5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1042 (Thread 0x7f91f6d23640 (LWP 2740976) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f6d1f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f6d1f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f6d1f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f6cedbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f6ced938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f6cee5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1041 (Thread 0x7f91f6f24640 (LWP 2740975) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f6f20750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f6f20750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f6f20730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f6eeebe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f6eee938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f6eef5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1040 (Thread 0x7f91f7125640 (LWP 2740974) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7121750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7121750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7121730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f70efbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f70ef938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f70f05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1039 (Thread 0x7f91f7326640 (LWP 2740973) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7322750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7322750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7322730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f72f0be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f72f0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f72f15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1038 (Thread 0x7f91f7527640 (LWP 2740972) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7523750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7523750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7523730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f74f1be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f74f1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f74f25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1037 (Thread 0x7f91f7728640 (LWP 2740971) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7724750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7724750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7724730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f76f2be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f76f2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f76f35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1036 (Thread 0x7f91f7929640 (LWP 2740970) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7925750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7925750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7925730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f78f3be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f78f3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f78f45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1035 (Thread 0x7f91f7b2a640 (LWP 2740969) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7b26750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7b26750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7b26730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f7af4be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f7af4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f7af55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1034 (Thread 0x7f91f7d2b640 (LWP 2740968) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7d27750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7d27750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7d27730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f7cf5be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f7cf5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f7cf65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1033 (Thread 0x7f91f7f2c640 (LWP 2740967) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f7f28750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f7f28750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f7f28730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f7ef6be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f7ef6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f7ef75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1032 (Thread 0x7f91f812d640 (LWP 2740966) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f8129750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f8129750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f8129730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f80f7be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f80f7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f80f85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1031 (Thread 0x7f91f832e640 (LWP 2740965) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f832a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f832a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f832a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f82f8be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f82f8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f82f95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1030 (Thread 0x7f91f852f640 (LWP 2740964) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f852b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f852b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f852b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f84f9be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f84f9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f84fa5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1029 (Thread 0x7f91f8730640 (LWP 2740963) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f872c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f872c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f872c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f86fabe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f86fa938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f86fb5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1028 (Thread 0x7f91f8931640 (LWP 2740962) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f892d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f892d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f892d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f88fbbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f88fb938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f88fc5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1027 (Thread 0x7f91f8b32640 (LWP 2740961) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f8b2e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f8b2e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f8b2e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f8afcbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f8afc938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f8afd5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1026 (Thread 0x7f91f8d33640 (LWP 2740960) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f8d2f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f8d2f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f8d2f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f8cfdbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f8cfd938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f8cfe5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1025 (Thread 0x7f91f8f34640 (LWP 2740959) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f8f30750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f8f30750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f8f30730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f8efebe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f8efe938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f8eff5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1024 (Thread 0x7f91f9135640 (LWP 2740958) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9131750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9131750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9131730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f90ffbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f90ff938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f91005d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1023 (Thread 0x7f91f9336640 (LWP 2740957) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9332750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9332750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9332730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9300be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9300938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f93015d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1022 (Thread 0x7f91f9537640 (LWP 2740956) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9533750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9533750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9533730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9501be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9501938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f95025d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1021 (Thread 0x7f91f9738640 (LWP 2740955) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9734750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9734750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9734730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9702be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9702938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f97035d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1020 (Thread 0x7f91f9939640 (LWP 2740954) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9935750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9935750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9935730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9903be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9903938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f99045d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1019 (Thread 0x7f91f9b3a640 (LWP 2740953) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9b36750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9b36750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9b36730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9b04be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9b04938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f9b055d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1018 (Thread 0x7f91f9d3b640 (LWP 2740952) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9d37750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9d37750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9d37730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9d05be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9d05938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f9d065d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1017 (Thread 0x7f91f9f3c640 (LWP 2740951) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91f9f38750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91f9f38750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91f9f38730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91f9f06be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91f9f06938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91f9f075d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1016 (Thread 0x7f91fa13d640 (LWP 2740950) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fa139750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fa139750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fa139730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fa107be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fa107938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fa1085d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1015 (Thread 0x7f91fa33e640 (LWP 2740949) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fa33a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fa33a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fa33a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fa308be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fa308938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fa3095d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1014 (Thread 0x7f91fa53f640 (LWP 2740948) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fa53b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fa53b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fa53b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fa509be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fa509938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fa50a5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1013 (Thread 0x7f91fa740640 (LWP 2740947) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fa73c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fa73c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fa73c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fa70abe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fa70a938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fa70b5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1012 (Thread 0x7f91fa941640 (LWP 2740946) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fa93d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fa93d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fa93d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fa90bbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fa90b938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fa90c5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1011 (Thread 0x7f91fab42640 (LWP 2740945) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fab3e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fab3e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fab3e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fab0cbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fab0c938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fab0d5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1010 (Thread 0x7f91fad43640 (LWP 2740944) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fad3f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fad3f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fad3f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fad0dbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fad0d938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fad0e5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1009 (Thread 0x7f91faf44640 (LWP 2740943) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91faf40750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91faf40750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91faf40730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91faf0ebe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91faf0e938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91faf0f5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1008 (Thread 0x7f91fb145640 (LWP 2740942) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fb141750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fb141750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fb141730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fb10fbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fb10f938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fb1105d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1007 (Thread 0x7f91fb346640 (LWP 2740941) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fb342750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fb342750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fb342730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fb310be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fb310938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fb3115d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1006 (Thread 0x7f91fb547640 (LWP 2740940) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fb543750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fb543750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fb543730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fb511be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fb511938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fb5125d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1005 (Thread 0x7f91fb748640 (LWP 2740939) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fb744750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fb744750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fb744730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fb712be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fb712938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fb7135d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1004 (Thread 0x7f91fb949640 (LWP 2740938) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fb945750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fb945750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fb945730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fb913be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fb913938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fb9145d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1003 (Thread 0x7f91fbb4a640 (LWP 2740937) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fbb46750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fbb46750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fbb46730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fbb14be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fbb14938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fbb155d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1002 (Thread 0x7f91fbd4b640 (LWP 2740936) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fbd47750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fbd47750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fbd47730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fbd15be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fbd15938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fbd165d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1001 (Thread 0x7f91fbf4c640 (LWP 2740935) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fbf48750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fbf48750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fbf48730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fbf16be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fbf16938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fbf175d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1000 (Thread 0x7f91fc14d640 (LWP 2740934) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fc149750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fc149750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fc149730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fc117be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fc117938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fc1185d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 999 (Thread 0x7f91fc34e640 (LWP 2740933) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fc34a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fc34a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fc34a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fc318be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fc318938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fc3195d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 998 (Thread 0x7f91fc54f640 (LWP 2740932) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fc54b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fc54b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fc54b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fc519be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fc519938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fc51a5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 997 (Thread 0x7f91fc750640 (LWP 2740931) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fc74c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fc74c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fc74c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fc71abe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fc71a938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fc71b5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 996 (Thread 0x7f91fc951640 (LWP 2740930) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fc94d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fc94d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fc94d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fc91bbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fc91b938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fc91c5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 995 (Thread 0x7f91fcb52640 (LWP 2740929) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fcb4e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fcb4e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fcb4e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fcb1cbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fcb1c938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fcb1d5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 994 (Thread 0x7f91fcd53640 (LWP 2740928) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fcd4f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fcd4f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fcd4f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fcd1dbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fcd1d938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fcd1e5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 993 (Thread 0x7f91fcf54640 (LWP 2740927) "sched-worker-pr"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fcf50750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fcf50750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fcf50730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247611120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fcf1ebe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e835 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f91fcf1e938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f3b6 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa244 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ae77 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a704 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9e0854 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be16f20 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f91fcf1f5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bd9c818 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb791c2 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67f1ef in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 992 (Thread 0x7f91fd155640 (LWP 2740922) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fd151750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fd151750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fd151730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fd120b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fd120a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fd121090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 991 (Thread 0x7f91fd356640 (LWP 2740921) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fd352750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fd352750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fd352730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fd321b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fd321a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fd322090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 990 (Thread 0x7f91fd557640 (LWP 2740919) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fd553750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fd553750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fd553730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fd522b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fd522a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fd523090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 989 (Thread 0x7f91fd758640 (LWP 2740917) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fd754750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fd754750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fd754730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fd723b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fd723a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fd724090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 988 (Thread 0x7f91fd959640 (LWP 2740915) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fd955750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fd955750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fd955730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fd924b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fd924a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fd925090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 987 (Thread 0x7f91fdb5a640 (LWP 2740914) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fdb56750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fdb56750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fdb56730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fdb25b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fdb25a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fdb26090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 986 (Thread 0x7f91fdd5b640 (LWP 2740913) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fdd57750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fdd57750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fdd57730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fdd26b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fdd26a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fdd27090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 985 (Thread 0x7f91fdf5c640 (LWP 2740912) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fdf58750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fdf58750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fdf58730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fdf27b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fdf27a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fdf28090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 984 (Thread 0x7f91fe15d640 (LWP 2740911) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fe159750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fe159750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fe159730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fe128b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fe128a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fe129090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 983 (Thread 0x7f91fe35e640 (LWP 2740910) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fe35a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fe35a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fe35a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fe329b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fe329a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fe32a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 982 (Thread 0x7f91fe55f640 (LWP 2740909) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fe55b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fe55b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fe55b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fe52ab30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fe52aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fe52b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 981 (Thread 0x7f91fe760640 (LWP 2740908) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fe75c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fe75c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fe75c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fe72bb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fe72ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fe72c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 980 (Thread 0x7f91fe961640 (LWP 2740907) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fe95d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fe95d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fe95d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fe92cb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fe92ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fe92d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 979 (Thread 0x7f91feb62640 (LWP 2740905) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91feb5e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91feb5e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91feb5e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91feb2db30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91feb2da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91feb2e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 978 (Thread 0x7f91fed63640 (LWP 2740903) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fed5f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fed5f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fed5f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fed2eb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fed2ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fed2f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 977 (Thread 0x7f91fef64640 (LWP 2740901) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fef60750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fef60750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fef60730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fef2fb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fef2fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fef30090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 976 (Thread 0x7f91ff165640 (LWP 2740900) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ff161750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ff161750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ff161730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ff130b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ff130a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ff131090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 975 (Thread 0x7f91ff366640 (LWP 2740899) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ff362750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ff362750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ff362730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ff331b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ff331a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ff332090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 974 (Thread 0x7f91ff567640 (LWP 2740898) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ff563750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ff563750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ff563730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ff532b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ff532a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ff533090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 973 (Thread 0x7f91ff768640 (LWP 2740897) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ff764750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ff764750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ff764730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ff733b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ff733a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ff734090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 972 (Thread 0x7f91ff969640 (LWP 2740896) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ff965750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ff965750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ff965730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ff934b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ff934a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ff935090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 971 (Thread 0x7f91ffb6a640 (LWP 2740895) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ffb66750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ffb66750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ffb66730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ffb35b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ffb35a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ffb36090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 970 (Thread 0x7f91ffd6b640 (LWP 2740894) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91ffd67750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91ffd67750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91ffd67730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91ffd36b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91ffd36a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91ffd37090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 969 (Thread 0x7f91fff6c640 (LWP 2740892) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f91fff68750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f91fff68750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f91fff68730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f91fff37b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f91fff37a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f91fff38090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 968 (Thread 0x7f920016d640 (LWP 2740891) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9200169750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9200169750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9200169730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9200138b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9200138a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9200139090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 967 (Thread 0x7f920036e640 (LWP 2740890) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920036a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920036a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920036a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9200339b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9200339a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920033a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 966 (Thread 0x7f920056f640 (LWP 2740889) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920056b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920056b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920056b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920053ab30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920053aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920053b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 965 (Thread 0x7f9200770640 (LWP 2740888) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920076c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920076c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920076c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920073bb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920073ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920073c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 964 (Thread 0x7f9200971640 (LWP 2740886) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920096d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920096d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920096d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920093cb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920093ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920093d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 963 (Thread 0x7f9200b72640 (LWP 2740885) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9200b6e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9200b6e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9200b6e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9200b3db30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9200b3da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9200b3e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 962 (Thread 0x7f9200d73640 (LWP 2740884) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9200d6f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9200d6f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9200d6f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9200d3eb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9200d3ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9200d3f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 961 (Thread 0x7f9200f74640 (LWP 2740883) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9200f70750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9200f70750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9200f70730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9200f3fb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9200f3fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9200f40090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 960 (Thread 0x7f9201175640 (LWP 2740882) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201171750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201171750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201171730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201140b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201140a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201141090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 959 (Thread 0x7f9201376640 (LWP 2740881) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201372750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201372750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201372730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201341b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201341a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201342090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 958 (Thread 0x7f9201577640 (LWP 2740880) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201573750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201573750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201573730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201542b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201542a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201543090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 957 (Thread 0x7f9201778640 (LWP 2740879) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201774750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201774750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201774730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201743b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201743a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201744090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 956 (Thread 0x7f9201979640 (LWP 2740877) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201975750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201975750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201975730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201944b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201944a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201945090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 955 (Thread 0x7f9201b7a640 (LWP 2740876) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201b76750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201b76750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201b76730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201b45b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201b45a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201b46090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 954 (Thread 0x7f9201d7b640 (LWP 2740875) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201d77750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201d77750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201d77730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201d46b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201d46a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201d47090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 953 (Thread 0x7f9201f7c640 (LWP 2740874) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9201f78750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9201f78750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9201f78730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9201f47b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9201f47a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9201f48090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 952 (Thread 0x7f920217d640 (LWP 2740873) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9202179750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9202179750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9202179730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9202148b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9202148a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9202149090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 951 (Thread 0x7f920237e640 (LWP 2740872) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920237a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920237a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920237a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9202349b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9202349a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920234a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 950 (Thread 0x7f920257f640 (LWP 2740870) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920257b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920257b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920257b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920254ab30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920254aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920254b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 949 (Thread 0x7f9202780640 (LWP 2740869) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920277c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920277c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920277c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920274bb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920274ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920274c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 948 (Thread 0x7f9202981640 (LWP 2740868) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920297d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920297d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920297d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920294cb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920294ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920294d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 947 (Thread 0x7f9202b82640 (LWP 2740867) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9202b7e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9202b7e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9202b7e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9202b4db30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9202b4da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9202b4e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 946 (Thread 0x7f9202d83640 (LWP 2740866) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9202d7f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9202d7f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9202d7f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9202d4eb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9202d4ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9202d4f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 945 (Thread 0x7f9202f84640 (LWP 2740865) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9202f80750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9202f80750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9202f80730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9202f4fb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9202f4fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9202f50090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 944 (Thread 0x7f9203185640 (LWP 2740864) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203181750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203181750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203181730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203150b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203150a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203151090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 943 (Thread 0x7f9203386640 (LWP 2740863) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203382750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203382750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203382730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203351b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203351a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203352090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 942 (Thread 0x7f9203587640 (LWP 2740861) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203583750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203583750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203583730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203552b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203552a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203553090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 941 (Thread 0x7f9203788640 (LWP 2740860) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203784750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203784750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203784730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203753b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203753a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203754090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 940 (Thread 0x7f9203989640 (LWP 2740859) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203985750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203985750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203985730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203954b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203954a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203955090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 939 (Thread 0x7f9203b8a640 (LWP 2740858) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203b86750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203b86750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203b86730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203b55b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203b55a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203b56090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 938 (Thread 0x7f9203d8b640 (LWP 2740857) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203d87750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203d87750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203d87730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203d56b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203d56a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203d57090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 937 (Thread 0x7f9203f8c640 (LWP 2740856) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9203f88750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9203f88750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9203f88730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9203f57b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9203f57a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9203f58090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 936 (Thread 0x7f920418d640 (LWP 2740855) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9204189750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9204189750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9204189730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9204158b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9204158a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9204159090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 935 (Thread 0x7f920438e640 (LWP 2740853) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920438a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920438a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920438a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9204359b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9204359a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920435a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 934 (Thread 0x7f920458f640 (LWP 2740852) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920458b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920458b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920458b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920455ab30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920455aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920455b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 933 (Thread 0x7f9204790640 (LWP 2740851) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920478c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920478c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920478c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920475bb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920475ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920475c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 932 (Thread 0x7f9204991640 (LWP 2740849) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920498d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920498d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920498d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920495cb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920495ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920495d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 931 (Thread 0x7f9204b92640 (LWP 2740848) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9204b8e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9204b8e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9204b8e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9204b5db30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9204b5da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9204b5e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 930 (Thread 0x7f9204d93640 (LWP 2740847) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9204d8f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9204d8f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9204d8f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9204d5eb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9204d5ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9204d5f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 929 (Thread 0x7f9204f94640 (LWP 2740846) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9204f90750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9204f90750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9204f90730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9204f5fb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9204f5fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9204f60090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 928 (Thread 0x7f9205195640 (LWP 2740845) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205191750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205191750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205191730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205160b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205160a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205161090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 927 (Thread 0x7f9205396640 (LWP 2740844) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205392750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205392750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205392730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205361b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205361a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205362090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 926 (Thread 0x7f9205597640 (LWP 2740843) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205593750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205593750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205593730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205562b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205562a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205563090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 925 (Thread 0x7f9205798640 (LWP 2740842) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205794750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205794750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205794730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205763b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205763a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205764090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 924 (Thread 0x7f9205999640 (LWP 2740841) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205995750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205995750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205995730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205964b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205964a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205965090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 923 (Thread 0x7f9205b9a640 (LWP 2740840) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205b96750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205b96750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205b96730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205b65b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205b65a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205b66090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 922 (Thread 0x7f9205d9b640 (LWP 2740839) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205d97750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205d97750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205d97730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205d66b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205d66a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205d67090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 921 (Thread 0x7f9205f9c640 (LWP 2740838) "sched-worker-hi"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9205f98750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9205f98750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9205f98730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247608080, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9205f67b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9205f67a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9205f68090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 920 (Thread 0x7f920619d640 (LWP 2740836) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9206199750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9206199750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9206199730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9206168b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9206168a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9206169090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 919 (Thread 0x7f920639e640 (LWP 2740835) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920639a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920639a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920639a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9206369b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9206369a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920636a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 918 (Thread 0x7f920659f640 (LWP 2740834) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920659b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920659b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920659b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920656ab30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920656aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920656b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 917 (Thread 0x7f92067a0640 (LWP 2740833) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920679c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920679c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920679c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920676bb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920676ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920676c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 916 (Thread 0x7f92069a1640 (LWP 2740832) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920699d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920699d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920699d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920696cb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920696ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920696d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 915 (Thread 0x7f9206ba2640 (LWP 2740831) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9206b9e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9206b9e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9206b9e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9206b6db30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9206b6da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9206b6e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 914 (Thread 0x7f9206da3640 (LWP 2740830) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9206d9f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9206d9f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9206d9f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9206d6eb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9206d6ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9206d6f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 913 (Thread 0x7f9206fa4640 (LWP 2740829) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9206fa0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9206fa0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9206fa0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9206f6fb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9206f6fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9206f70090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 912 (Thread 0x7f92071a5640 (LWP 2740828) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92071a1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92071a1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92071a1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207170b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207170a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207171090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 911 (Thread 0x7f92073a6640 (LWP 2740827) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92073a2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92073a2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92073a2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207371b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207371a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207372090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 910 (Thread 0x7f92075a7640 (LWP 2740826) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92075a3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92075a3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92075a3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207572b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207572a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207573090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 909 (Thread 0x7f92077a8640 (LWP 2740825) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92077a4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92077a4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92077a4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207773b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207773a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207774090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 908 (Thread 0x7f92079a9640 (LWP 2740824) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92079a5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92079a5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92079a5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207974b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207974a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207975090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 907 (Thread 0x7f9207baa640 (LWP 2740823) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9207ba6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9207ba6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9207ba6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207b75b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207b75a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207b76090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 906 (Thread 0x7f9207dab640 (LWP 2740822) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9207da7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9207da7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9207da7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207d76b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207d76a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207d77090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 905 (Thread 0x7f9207fac640 (LWP 2740821) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9207fa8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9207fa8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9207fa8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9207f77b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9207f77a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9207f78090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 904 (Thread 0x7f92081ad640 (LWP 2740820) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92081a9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92081a9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92081a9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9208178b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9208178a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9208179090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 903 (Thread 0x7f92083ae640 (LWP 2740819) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92083aa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92083aa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92083aa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9208379b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9208379a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920837a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 902 (Thread 0x7f92085af640 (LWP 2740818) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92085ab750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92085ab750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92085ab730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920857ab30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920857aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920857b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 901 (Thread 0x7f92087b0640 (LWP 2740817) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92087ac750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92087ac750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92087ac730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920877bb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920877ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920877c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 900 (Thread 0x7f92089b1640 (LWP 2740816) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92089ad750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92089ad750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92089ad730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920897cb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920897ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920897d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 899 (Thread 0x7f9208bb2640 (LWP 2740815) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9208bae750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9208bae750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9208bae730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9208b7db30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9208b7da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9208b7e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 898 (Thread 0x7f9208db3640 (LWP 2740814) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9208daf750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9208daf750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9208daf730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9208d7eb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9208d7ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9208d7f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 897 (Thread 0x7f9208fb4640 (LWP 2740813) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9208fb0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9208fb0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9208fb0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9208f7fb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9208f7fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9208f80090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 896 (Thread 0x7f92091b5640 (LWP 2740812) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92091b1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92091b1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92091b1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209180b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209180a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209181090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 895 (Thread 0x7f92093b6640 (LWP 2740811) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92093b2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92093b2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92093b2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209381b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209381a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209382090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 894 (Thread 0x7f92095b7640 (LWP 2740810) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92095b3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92095b3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92095b3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209582b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209582a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209583090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 893 (Thread 0x7f92097b8640 (LWP 2740809) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92097b4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92097b4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92097b4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209783b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209783a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209784090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 892 (Thread 0x7f92099b9640 (LWP 2740808) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92099b5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92099b5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92099b5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209984b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209984a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209985090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 891 (Thread 0x7f9209bba640 (LWP 2740807) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9209bb6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9209bb6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9209bb6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209b85b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209b85a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209b86090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 890 (Thread 0x7f9209dbb640 (LWP 2740806) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9209db7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9209db7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9209db7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209d86b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209d86a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209d87090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 889 (Thread 0x7f9209fbc640 (LWP 2740805) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9209fb8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9209fb8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9209fb8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9209f87b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f9209f87a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f9209f88090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 888 (Thread 0x7f920a1bd640 (LWP 2740804) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920a1b9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920a1b9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920a1b9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920a188b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920a188a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920a189090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 887 (Thread 0x7f920a3be640 (LWP 2740803) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920a3ba750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920a3ba750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920a3ba730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920a389b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920a389a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920a38a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 886 (Thread 0x7f920a5bf640 (LWP 2740802) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920a5bb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920a5bb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920a5bb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920a58ab30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920a58aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920a58b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 885 (Thread 0x7f920a7c0640 (LWP 2740801) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920a7bc750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920a7bc750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920a7bc730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920a78bb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920a78ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920a78c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 884 (Thread 0x7f920a9c1640 (LWP 2740800) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920a9bd750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920a9bd750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920a9bd730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920a98cb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920a98ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920a98d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 883 (Thread 0x7f920abc2640 (LWP 2740799) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920abbe750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920abbe750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920abbe730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920ab8db30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920ab8da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920ab8e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 882 (Thread 0x7f920adc3640 (LWP 2740798) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920adbf750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920adbf750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920adbf730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920ad8eb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920ad8ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920ad8f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 881 (Thread 0x7f920afc4640 (LWP 2740797) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920afc0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920afc0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920afc0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920af8fb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920af8fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920af90090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 880 (Thread 0x7f920b1c5640 (LWP 2740796) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920b1c1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920b1c1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920b1c1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920b190b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920b190a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920b191090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 879 (Thread 0x7f920b3c6640 (LWP 2740795) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920b3c2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920b3c2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920b3c2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920b391b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920b391a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920b392090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 878 (Thread 0x7f920b5c7640 (LWP 2740794) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920b5c3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920b5c3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920b5c3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920b592b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920b592a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920b593090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 877 (Thread 0x7f920b7c8640 (LWP 2740793) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920b7c4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920b7c4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920b7c4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920b793b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920b793a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920b794090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 876 (Thread 0x7f920b9c9640 (LWP 2740792) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920b9c5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920b9c5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920b9c5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920b994b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920b994a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920b995090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 875 (Thread 0x7f920bbca640 (LWP 2740791) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920bbc6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920bbc6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920bbc6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920bb95b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920bb95a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920bb96090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 874 (Thread 0x7f920bdcb640 (LWP 2740790) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920bdc7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920bdc7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920bdc7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920bd96b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920bd96a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920bd97090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 873 (Thread 0x7f920bfcc640 (LWP 2740789) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920bfc8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920bfc8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920bfc8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920bf97b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920bf97a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920bf98090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 872 (Thread 0x7f920c1cd640 (LWP 2740788) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920c1c9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920c1c9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920c1c9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920c198b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920c198a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920c199090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 871 (Thread 0x7f920c3ce640 (LWP 2740787) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920c3ca750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920c3ca750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920c3ca730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920c399b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920c399a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920c39a090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 870 (Thread 0x7f920c5cf640 (LWP 2740786) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920c5cb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920c5cb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920c5cb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920c59ab30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920c59aa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920c59b090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 869 (Thread 0x7f920c7d0640 (LWP 2740785) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920c7cc750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920c7cc750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920c7cc730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920c79bb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920c79ba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920c79c090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 868 (Thread 0x7f920c9d1640 (LWP 2740784) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920c9cd750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920c9cd750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920c9cd730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920c99cb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920c99ca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920c99d090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 867 (Thread 0x7f920cbd2640 (LWP 2740783) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920cbce750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920cbce750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920cbce730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920cb9db30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920cb9da10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920cb9e090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 866 (Thread 0x7f920cdd3640 (LWP 2740782) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920cdcf750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920cdcf750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920cdcf730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920cd9eb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920cd9ea10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920cd9f090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 865 (Thread 0x7f920cfd4640 (LWP 2740781) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920cfd0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920cfd0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920cfd0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920cf9fb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920cf9fa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920cfa0090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 864 (Thread 0x7f920d1d5640 (LWP 2740780) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920d1d1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920d1d1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920d1d1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920d1a0b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920d1a0a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920d1a1090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 863 (Thread 0x7f920d3d6640 (LWP 2740779) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920d3d2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920d3d2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920d3d2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920d3a1b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920d3a1a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920d3a2090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 862 (Thread 0x7f920d5d7640 (LWP 2740778) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920d5d3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920d5d3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920d5d3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920d5a2b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920d5a2a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920d5a3090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 861 (Thread 0x7f920d7d8640 (LWP 2740777) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920d7d4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920d7d4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920d7d4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920d7a3b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920d7a3a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920d7a4090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 860 (Thread 0x7f920d9d9640 (LWP 2740776) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920d9d5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920d9d5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920d9d5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920d9a4b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920d9a4a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920d9a5090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 859 (Thread 0x7f920dbda640 (LWP 2740775) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920dbd6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920dbd6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920dbd6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920dba5b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920dba5a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920dba6090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 858 (Thread 0x7f920dddb640 (LWP 2740774) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920ddd7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920ddd7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920ddd7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920dda6b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920dda6a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920dda7090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 857 (Thread 0x7f920dfdc640 (LWP 2740773) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920dfd8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920dfd8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920dfd8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920dfa7b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920dfa7a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920dfa8090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 856 (Thread 0x7f920e1dd640 (LWP 2740772) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920e1d9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920e1d9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920e1d9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920e1a8b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920e1a8a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920e1a9090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 855 (Thread 0x7f920e3de640 (LWP 2740771) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920e3da750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920e3da750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920e3da730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920e3a9b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920e3a9a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920e3aa090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 854 (Thread 0x7f920e5df640 (LWP 2740770) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920e5db750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920e5db750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920e5db730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920e5aab30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920e5aaa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920e5ab090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 853 (Thread 0x7f920e7e0640 (LWP 2740769) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920e7dc750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920e7dc750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920e7dc730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920e7abb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920e7aba10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920e7ac090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 852 (Thread 0x7f920e9e1640 (LWP 2740768) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920e9dd750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920e9dd750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920e9dd730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920e9acb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920e9aca10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920e9ad090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 851 (Thread 0x7f920ebe2640 (LWP 2740767) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f920ebde750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f920ebde750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f920ebde730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f920ebadb30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f920ebada10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f920ebae090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 850 (Thread 0x7f94afffe640 (LWP 2740766) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94afffa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94afffa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94afffa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94affc9b30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f94affc9a10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f94affca090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 849 (Thread 0x7f94b19ff640 (LWP 2740765) "sched-worker-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94b19fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94b19fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94b19fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247607120, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94b19cab30) at src/pool/spawn.rs:305 -#7 0x00007f957d08e9c5 in yatp::pool::worker::WorkerThread>>>>::pop>>>> (self=0x7f94b19caa10) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08f876 in yatp::pool::worker::WorkerThread>>>>::run>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa2a0 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80add7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a460 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9d7664 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be177f4 in std::panicking::try::do_call>>>>>, ()>>, ()> (data=0x7f94b19cb090) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bdb0d48 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb789d0 in std::panic::catch_unwind>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d67d67f in core::ops::function::FnOnce::call_once>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 848 (Thread 0x7f94b21ff640 (LWP 2740759) "resource-meteri"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94b21fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94b21fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94b21fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689433552, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94b21cac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94b21cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94b21cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 847 (Thread 0x7f94b3dff640 (LWP 2740758) "resource-meteri"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94b3dfb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94b3dfb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94b3dfb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140277123436240, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94b3dcac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94b3dcab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94b3dcb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 846 (Thread 0x7f94b55ff640 (LWP 2740757) "resource-meteri"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94b55fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94b55fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94b55fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140266247604720, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94b55cac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94b55cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94b55cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 845 (Thread 0x7f94b77ff640 (LWP 2740756) "debugger"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583198147 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94b77fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f9583197f74 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94b77fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f958319261e in parking_lot_core::parking_lot::park::{closure#0} (thread_data=0x7f94b77fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f958319095f in parking_lot_core::parking_lot::with_thread_data> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park (key=140279689410680, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95831958a9 in parking_lot::condvar::Condvar::wait_until_internal (self=0x7f95690e7878, mutex=0x7f95690e7880, timeout=...) at src/condvar.rs:333 -#7 0x00007f95830d11de in parking_lot::condvar::Condvar::wait<()> (self=0x7f95690e7878, mutex_guard=0x7f94b77ca118) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot-0.12.1/src/condvar.rs:256 -#8 0x00007f95831658cc in tokio::loom::std::parking_lot::Condvar::wait<()> (self=0x7f95690e7878, guard=...) at src/loom/std/parking_lot.rs:149 -#9 0x00007f95830ef2f6 in tokio::runtime::park::Inner::park (self=0x7f95690e7870) at src/runtime/park.rs:116 -#10 0x00007f95830ef113 in tokio::runtime::park::ParkThread::park (self=0x7f9561594ea0) at src/runtime/park.rs:62 -#11 0x00007f95830f1adf in tokio::runtime::driver::IoStack::park (self=0x7f9561594e98, handle=0x7f9561584d18) at src/runtime/driver.rs:176 -#12 0x00007f958316a02e in tokio::runtime::time::Driver::park_internal (self=0x7f9561594e98, rt_handle=0x7f9561584d18, limit=...) at src/runtime/time/mod.rs:235 -#13 0x00007f9583169be4 in tokio::runtime::time::Driver::park (self=0x7f9561594e98, handle=0x7f9561584d18) at src/runtime/time/mod.rs:165 -#14 0x00007f95830f2076 in tokio::runtime::driver::TimeDriver::park (self=0x7f9561594e90, handle=0x7f9561584d18) at src/runtime/driver.rs:332 -#15 0x00007f95830f0873 in tokio::runtime::driver::Driver::park (self=0x7f9561594e90, handle=0x7f9561584d18) at src/runtime/driver.rs:71 -#16 0x00007f95830f30e2 in tokio::runtime::scheduler::multi_thread::park::Inner::park_driver (self=0x7f95690e7930, driver=0x7f9561594e90, handle=0x7f9561584d18) at src/runtime/scheduler/multi_thread/park.rs:184 -#17 0x00007f95830f2b9c in tokio::runtime::scheduler::multi_thread::park::Inner::park (self=0x7f95690e7930, handle=0x7f9561584d18) at src/runtime/scheduler/multi_thread/park.rs:117 -#18 0x00007f95830f27a4 in tokio::runtime::scheduler::multi_thread::park::Parker::park (self=0x7f94b77ca9a0, handle=0x7f9561584d18) at src/runtime/scheduler/multi_thread/park.rs:67 -#19 0x00007f958311ad53 in tokio::runtime::scheduler::multi_thread::worker::Context::park_timeout (self=0x7f94b77cae98, core=0x7f9561598f90, duration=...) at src/runtime/scheduler/multi_thread/worker.rs:732 -#20 0x00007f958311a862 in tokio::runtime::scheduler::multi_thread::worker::Context::park (self=0x7f94b77cae98, core=0x7f9561598f90) at src/runtime/scheduler/multi_thread/worker.rs:702 -#21 0x00007f958311945f in tokio::runtime::scheduler::multi_thread::worker::Context::run (self=0x7f94b77cae98, core=0x7f9561598f90) at src/runtime/scheduler/multi_thread/worker.rs:553 -#22 0x00007f9583118d59 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:500 -#23 0x00007f9583151a70 in tokio::runtime::context::scoped::Scoped::set (self=0x7f94b77cc610, t=0x7f94b77cae90, f=...) at src/runtime/context/scoped.rs:40 -#24 0x00007f958310eaeb in tokio::runtime::context::set_scheduler::{closure#0}<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (c=0x7f94b77cc5d8) at src/runtime/context.rs:180 -#25 0x00007f9583155b52 in std::thread::local::LocalKey::try_with, ()> (self=0x7f95840cad88, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#26 0x00007f958315502b in std::thread::local::LocalKey::with, ()> (self=0x7f95840cad88, f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#27 0x00007f958310eab4 in tokio::runtime::context::set_scheduler<(), tokio::runtime::scheduler::multi_thread::worker::run::{closure#0}::{closure_env#0}> (v=0x7f94b77cae90, f=...) at src/runtime/context.rs:180 -#28 0x00007f9583118c61 in tokio::runtime::scheduler::multi_thread::worker::run::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:495 -#29 0x00007f9583151488 in tokio::runtime::context::runtime::enter_runtime (handle=0x7f94b77cb098, allow_block_in_place=true, f=...) at src/runtime/context/runtime.rs:65 -#30 0x00007f95831189fc in tokio::runtime::scheduler::multi_thread::worker::run (worker=...) at src/runtime/scheduler/multi_thread/worker.rs:487 -#31 0x00007f958311886b in tokio::runtime::scheduler::multi_thread::worker::{impl#0}::launch::{closure#0} () at src/runtime/scheduler/multi_thread/worker.rs:455 -#32 0x00007f95830fbdbe in tokio::runtime::blocking::task::{impl#2}::poll (self=..., _cx=0x7f94b77cb220) at src/runtime/blocking/task.rs:42 -#33 0x00007f958311f4bc in tokio::runtime::task::core::{impl#6}::poll::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=0x7f9561594e28) at src/runtime/task/core.rs:328 -#34 0x00007f958311ed2f in tokio::loom::std::unsafe_cell::UnsafeCell>>::with_mut>, core::task::poll::Poll<()>, tokio::runtime::task::core::{impl#6}::poll::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=0x7f9561594e28, f=...) at src/loom/std/unsafe_cell.rs:16 -#35 tokio::runtime::task::core::Core, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f9561594e20, cx=...) at src/runtime/task/core.rs:317 -#36 0x00007f9583124355 in tokio::runtime::task::harness::poll_future::{closure#0}, tokio::runtime::blocking::schedule::BlockingSchedule> () at src/runtime/task/harness.rs:485 -#37 0x00007f95830d0ec4 in core::panic::unwind_safe::{impl#23}::call_once, tokio::runtime::task::harness::poll_future::{closure_env#0}, tokio::runtime::blocking::schedule::BlockingSchedule>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#38 0x00007f9583158d56 in std::panicking::try::do_call, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (data=0x7f94b77cb398) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#39 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#40 0x00007f9583158058 in std::panicking::try, core::panic::unwind_safe::AssertUnwindSafe, tokio::runtime::blocking::schedule::BlockingSchedule>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#41 0x00007f9583154f3b in std::panic::catch_unwind, tokio::runtime::blocking::schedule::BlockingSchedule>>, core::task::poll::Poll<()>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#42 0x00007f95831238cf in tokio::runtime::task::harness::poll_future, tokio::runtime::blocking::schedule::BlockingSchedule> (core=0x7f9561594e20, cx=...) at src/runtime/task/harness.rs:473 -#43 0x00007f9583121b29 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll_inner, tokio::runtime::blocking::schedule::BlockingSchedule> (self=0x7f94b77cb5b0) at src/runtime/task/harness.rs:208 -#44 0x00007f9583121617 in tokio::runtime::task::harness::Harness, tokio::runtime::blocking::schedule::BlockingSchedule>::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (self=...) at src/runtime/task/harness.rs:153 -#45 0x00007f9583124f3d in tokio::runtime::task::raw::poll, tokio::runtime::blocking::schedule::BlockingSchedule> (ptr=...) at src/runtime/task/raw.rs:271 -#46 0x00007f9583124cc7 in tokio::runtime::task::raw::RawTask::poll (self=...) at src/runtime/task/raw.rs:201 -#47 0x00007f958316b2e7 in tokio::runtime::task::UnownedTask::run (self=...) at src/runtime/task/mod.rs:464 -#48 0x00007f958316b6c7 in tokio::runtime::blocking::pool::Task::run (self=...) at src/runtime/blocking/pool.rs:159 -#49 0x00007f958316e229 in tokio::runtime::blocking::pool::Inner::run (self=0x7f95615accd0, worker_thread_id=0) at src/runtime/blocking/pool.rs:513 -#50 0x00007f958316df64 in tokio::runtime::blocking::pool::{impl#6}::spawn_thread::{closure#0} () at src/runtime/blocking/pool.rs:471 -#51 0x00007f958312e616 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#52 0x00007f9583106932 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#53 0x00007f95830d0bc2 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#54 0x00007f95831590a3 in std::panicking::try::do_call>, ()> (data=0x7f94b77cba20) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#55 0x00007f958316fd9b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#56 0x00007f9583157ac1 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#57 0x00007f958310673f in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#58 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#59 0x00007f95830d2a8f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#60 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#61 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#62 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#63 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#64 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 844 (Thread 0x7f92145e3640 (LWP 2740755) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92145df750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92145df750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92145df730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92145adbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92145ad938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92145ae5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 843 (Thread 0x7f9214fe4640 (LWP 2740754) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9214fe0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9214fe0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9214fe0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9214faebe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9214fae938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9214faf5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 842 (Thread 0x7f92159e5640 (LWP 2740753) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92159e1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92159e1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92159e1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92159afbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92159af938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92159b05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 841 (Thread 0x7f92163e6640 (LWP 2740752) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92163e2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92163e2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92163e2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92163b0be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92163b0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92163b15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 840 (Thread 0x7f9216de7640 (LWP 2740751) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9216de3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9216de3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9216de3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9216db1be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9216db1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9216db25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 839 (Thread 0x7f92177e8640 (LWP 2740750) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92177e4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92177e4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92177e4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92177b2be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92177b2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92177b35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 838 (Thread 0x7f92181e9640 (LWP 2740749) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92181e5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92181e5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92181e5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92181b3be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92181b3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92181b45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 837 (Thread 0x7f9218bea640 (LWP 2740748) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9218be6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9218be6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9218be6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9218bb4be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9218bb4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9218bb55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 836 (Thread 0x7f92195eb640 (LWP 2740747) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92195e7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92195e7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92195e7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92195b5be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92195b5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92195b65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 835 (Thread 0x7f9219fec640 (LWP 2740746) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9219fe8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9219fe8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9219fe8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9219fb6be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9219fb6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9219fb75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 834 (Thread 0x7f921a9ed640 (LWP 2740745) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921a9e9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921a9e9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921a9e9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921a9b7be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921a9b7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921a9b85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 833 (Thread 0x7f921b3ee640 (LWP 2740744) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921b3ea750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921b3ea750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921b3ea730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921b3b8be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921b3b8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921b3b95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 832 (Thread 0x7f921bdef640 (LWP 2740743) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921bdeb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921bdeb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921bdeb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921bdb9be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921bdb9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921bdba5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 831 (Thread 0x7f921c7f0640 (LWP 2740742) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921c7ec750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921c7ec750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921c7ec730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921c7babe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921c7ba938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921c7bb5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 830 (Thread 0x7f921d1f1640 (LWP 2740741) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921d1ed750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921d1ed750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921d1ed730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921d1bbbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921d1bb938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921d1bc5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 829 (Thread 0x7f921dbf2640 (LWP 2740740) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921dbee750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921dbee750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921dbee730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921dbbcbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921dbbc938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921dbbd5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 828 (Thread 0x7f921e5f3640 (LWP 2740739) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921e5ef750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921e5ef750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921e5ef730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921e5bdbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921e5bd938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921e5be5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 827 (Thread 0x7f921eff4640 (LWP 2740738) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921eff0750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921eff0750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921eff0730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921efbebe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921efbe938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921efbf5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 826 (Thread 0x7f921f9f5640 (LWP 2740737) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f921f9f1750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f921f9f1750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f921f9f1730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f921f9bfbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f921f9bf938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f921f9c05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 825 (Thread 0x7f92203f6640 (LWP 2740736) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92203f2750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92203f2750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92203f2730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92203c0be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92203c0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92203c15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 824 (Thread 0x7f9220df7640 (LWP 2740735) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9220df3750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9220df3750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9220df3730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9220dc1be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9220dc1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9220dc25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 823 (Thread 0x7f92217f8640 (LWP 2740734) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92217f4750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92217f4750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92217f4730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92217c2be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92217c2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92217c35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 822 (Thread 0x7f92221f9640 (LWP 2740733) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92221f5750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92221f5750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92221f5730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92221c3be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92221c3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92221c45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 821 (Thread 0x7f9222bfa640 (LWP 2740732) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9222bf6750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9222bf6750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9222bf6730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9222bc4be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9222bc4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9222bc55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 820 (Thread 0x7f92235fb640 (LWP 2740731) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92235f7750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92235f7750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92235f7730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92235c5be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92235c5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92235c65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 819 (Thread 0x7f9223ffc640 (LWP 2740730) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9223ff8750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9223ff8750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9223ff8730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9223fc6be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9223fc6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9223fc75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 818 (Thread 0x7f92249fd640 (LWP 2740729) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92249f9750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92249f9750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92249f9730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92249c7be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92249c7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92249c85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 817 (Thread 0x7f92253fe640 (LWP 2740728) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92253fa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92253fa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92253fa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92253c8be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92253c8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92253c95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 816 (Thread 0x7f9225dff640 (LWP 2740727) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9225dfb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9225dfb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9225dfb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9225dc9be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9225dc9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9225dca5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 815 (Thread 0x7f9226800640 (LWP 2740726) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92267fc750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92267fc750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92267fc730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92267cabe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92267ca938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92267cb5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 814 (Thread 0x7f9227201640 (LWP 2740725) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92271fd750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92271fd750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92271fd730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92271cbbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92271cb938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92271cc5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 813 (Thread 0x7f9227c02640 (LWP 2740724) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9227bfe750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9227bfe750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9227bfe730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9227bccbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9227bcc938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9227bcd5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 812 (Thread 0x7f9228603640 (LWP 2740723) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f92285ff750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f92285ff750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f92285ff730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92285cdbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92285cd938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92285ce5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 811 (Thread 0x7f9229004640 (LWP 2740722) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9229000750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9229000750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9229000730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9228fcebe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9228fce938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9228fcf5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 810 (Thread 0x7f9229a05640 (LWP 2740721) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9229a01750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9229a01750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9229a01730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92299cfbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92299cf938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92299d05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 809 (Thread 0x7f922a406640 (LWP 2740720) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922a402750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922a402750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922a402730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922a3d0be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922a3d0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922a3d15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 808 (Thread 0x7f922ae07640 (LWP 2740719) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922ae03750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922ae03750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922ae03730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922add1be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922add1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922add25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 807 (Thread 0x7f922b808640 (LWP 2740718) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922b804750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922b804750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922b804730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922b7d2be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922b7d2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922b7d35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 806 (Thread 0x7f922c209640 (LWP 2740717) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922c205750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922c205750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922c205730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922c1d3be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922c1d3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922c1d45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 805 (Thread 0x7f922cc0a640 (LWP 2740716) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922cc06750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922cc06750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922cc06730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922cbd4be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922cbd4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922cbd55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 804 (Thread 0x7f922d60b640 (LWP 2740715) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922d607750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922d607750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922d607730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922d5d5be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922d5d5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922d5d65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 803 (Thread 0x7f922e00c640 (LWP 2740714) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922e008750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922e008750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922e008730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922dfd6be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922dfd6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922dfd75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 802 (Thread 0x7f922ea0d640 (LWP 2740713) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922ea09750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922ea09750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922ea09730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922e9d7be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922e9d7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922e9d85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 801 (Thread 0x7f922f40e640 (LWP 2740712) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922f40a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922f40a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922f40a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922f3d8be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922f3d8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922f3d95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 800 (Thread 0x7f922fe0f640 (LWP 2740711) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f922fe0b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f922fe0b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f922fe0b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f922fdd9be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f922fdd9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f922fdda5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 799 (Thread 0x7f9230810640 (LWP 2740710) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923080c750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923080c750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923080c730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92307dabe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92307da938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92307db5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 798 (Thread 0x7f9231211640 (LWP 2740709) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923120d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923120d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923120d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92311dbbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92311db938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92311dc5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 797 (Thread 0x7f9231c12640 (LWP 2740708) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9231c0e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9231c0e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9231c0e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9231bdcbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9231bdc938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9231bdd5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 796 (Thread 0x7f9232613640 (LWP 2740707) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923260f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923260f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923260f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92325ddbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92325dd938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92325de5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 795 (Thread 0x7f9233014640 (LWP 2740706) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9233010750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9233010750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9233010730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9232fdebe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9232fde938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9232fdf5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 794 (Thread 0x7f9233a15640 (LWP 2740705) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9233a11750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9233a11750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9233a11730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92339dfbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92339df938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92339e05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 793 (Thread 0x7f9234416640 (LWP 2740704) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9234412750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9234412750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9234412730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92343e0be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92343e0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92343e15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 792 (Thread 0x7f9234e17640 (LWP 2740703) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9234e13750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9234e13750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9234e13730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9234de1be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9234de1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9234de25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 791 (Thread 0x7f9235818640 (LWP 2740702) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9235814750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9235814750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9235814730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92357e2be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92357e2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92357e35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 790 (Thread 0x7f9236219640 (LWP 2740701) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9236215750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9236215750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9236215730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92361e3be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92361e3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92361e45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 789 (Thread 0x7f9236c1a640 (LWP 2740700) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9236c16750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9236c16750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9236c16730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9236be4be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9236be4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9236be55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 788 (Thread 0x7f923761b640 (LWP 2740699) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9237617750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9237617750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9237617730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92375e5be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92375e5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92375e65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 787 (Thread 0x7f923801c640 (LWP 2740698) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9238018750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9238018750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9238018730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9237fe6be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9237fe6938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9237fe75d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 786 (Thread 0x7f9238a1d640 (LWP 2740697) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9238a19750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9238a19750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9238a19730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92389e7be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92389e7938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92389e85d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 785 (Thread 0x7f923941e640 (LWP 2740696) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923941a750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923941a750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923941a730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92393e8be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92393e8938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92393e95d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 784 (Thread 0x7f9239e1f640 (LWP 2740695) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9239e1b750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9239e1b750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9239e1b730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9239de9be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9239de9938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9239dea5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 783 (Thread 0x7f923c623640 (LWP 2740694) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923c61f750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923c61f750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923c61f730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923c5edbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923c5ed938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923c5ee5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 782 (Thread 0x7f923d024640 (LWP 2740693) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923d020750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923d020750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923d020730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923cfeebe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923cfee938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923cfef5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 781 (Thread 0x7f923da25640 (LWP 2740692) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923da21750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923da21750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923da21730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923d9efbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923d9ef938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923d9f05d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 780 (Thread 0x7f923e426640 (LWP 2740691) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923e422750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923e422750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923e422730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923e3f0be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923e3f0938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923e3f15d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 779 (Thread 0x7f923ee27640 (LWP 2740690) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923ee23750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923ee23750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923ee23730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923edf1be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923edf1938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923edf25d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 778 (Thread 0x7f923f828640 (LWP 2740689) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923f824750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923f824750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923f824730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923f7f2be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923f7f2938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923f7f35d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 777 (Thread 0x7f9240229640 (LWP 2740688) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9240225750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9240225750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9240225730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92401f3be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92401f3938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92401f45d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 776 (Thread 0x7f9240c2a640 (LWP 2740687) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9240c26750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9240c26750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9240c26730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9240bf4be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f9240bf4938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f9240bf55d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 775 (Thread 0x7f924162b640 (LWP 2740686) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9241627750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9241627750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9241627730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f92415f5be0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f92415f5938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f92415f65d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 774 (Thread 0x7f923bc22640 (LWP 2740685) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923bc1e750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923bc1e750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923bc1e730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923bbecbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923bbec938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923bbed5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 773 (Thread 0x7f923b221640 (LWP 2740684) "unified-read-po"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f923b21d750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f923b21d750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f923b21d730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689437392, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f923b1ebbe0) at src/pool/spawn.rs:305 -#7 0x00007f957d08e385 in yatp::pool::worker::WorkerThread>>>>>::pop>>>>> (self=0x7f923b1eb938) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f957d08ec96 in yatp::pool::worker::WorkerThread>>>>>::run>>>>> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f957bfaa304 in yatp::pool::builder::{impl#3}::build::{closure#0}>>>>>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f957a80ada7 in std::sys_common::backtrace::__rust_begin_short_backtrace>>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f957cb7a9a4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f957c9de8d4 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f957be03130 in std::panicking::try::do_call>>>>>>, ()>>, ()> (data=0x7f923b1ec5d0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f957bda0f78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f957cb77172 in std::panic::catch_unwind>>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f957d680c5f in core::ops::function::FnOnce::call_once>>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 772 (Thread 0x7f94bc3fe640 (LWP 2740683) "pd-worker-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94bc3fa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94bc3fa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94bc3fa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689434352, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94bc3c9c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94bc3c9b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94bc3ca170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 771 (Thread 0x7f94c29ff640 (LWP 2740681) "gc-worker-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94c29fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94c29fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94c29fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140277122565072, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94c29cac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94c29cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94c29cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 770 (Thread 0x7f94c61ff640 (LWP 2740680) "flow-checker"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cfc2f4 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 0x00007f9583cea9cc in std::sys_common::thread_parking::futex::Parker::park_timeout () at library/std/src/sys_common/thread_parking/futex.rs:72 -#3 std::thread::park_timeout () at library/std/src/thread/mod.rs:1133 -#4 0x00007f957e13f3e0 in std::sync::mpmc::context::Context::wait_until (self=0x7f94c61ca498, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:130 -#5 0x00007f957e42fa4a in std::sync::mpmc::list::{impl#3}::recv::{closure#1} (cx=0x7f94c61ca498) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:444 -#6 0x00007f957e4349d0 in std::sync::mpmc::context::{impl#0}::with::{closure#0}, ()> (cx=, cx=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:50 -#7 std::sync::mpmc::context::{impl#0}::with::{closure#1}, ()> (cell=0x7f94c61fba38) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:58 -#8 0x00007f957e439c1b in std::thread::local::LocalKey>>::try_with>, std::sync::mpmc::context::{impl#0}::with::{closure_env#1}, ()>, ()> (self=0x7f9583da0378, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f957e4316de in std::sync::mpmc::context::Context::with, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:53 -#10 0x00007f957e42f4b1 in std::sync::mpmc::list::Channel::recv (self=0x7f956a81e600, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:434 -#11 0x00007f957e13e8fc in std::sync::mpmc::Receiver::recv_deadline (self=0x7f94c61cade0, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/mod.rs:340 -#12 0x00007f957dfe1ad0 in std::sync::mpsc::Receiver::recv_deadline (self=0x7f94c61cade0, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpsc/mod.rs:970 -#13 0x00007f957d07b0e3 in tikv::storage::txn::flow_controller::singleton_flow_controller::{impl#6}::start::{closure#0} () at src/storage/txn/flow_controller/singleton_flow_controller.rs:489 -#14 0x00007f957df427c9 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, ()> () at components/tikv_util/src/sys/thread.rs:441 -#15 0x00007f957a80ae97 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#16 0x00007f957cb7a790 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#17 0x00007f957c9e2b04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#18 0x00007f957bdee664 in std::panicking::try::do_call, ()>, ()>>, ()> (data=0x7f94c61cb250) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#19 0x00007f957c51a68b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#20 0x00007f957bdaee78 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#21 0x00007f957cb79a00 in std::panic::catch_unwind, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#22 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#23 0x00007f957d67eb9f in core::ops::function::FnOnce::call_once, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#24 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#25 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#26 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#27 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#28 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 769 (Thread 0x7f923a820640 (LWP 2740679) "RaftStoreProxy"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f957068996f in pthread_cond_timedwait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9582217915 in rocksdb::port::CondVar::TimedWait(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f95823a23e9 in rocksdb::InstrumentedCondVar::TimedWait(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f95825f0201 in rocksdb::Timer::Run() () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f95825f06f2 in void* std::__1::__thread_proxy[abi:ue170006] >, void (rocksdb::Timer::*)(), rocksdb::Timer*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 768 (Thread 0x7f924202c640 (LWP 2740611) "rocksdb:high"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 767 (Thread 0x7f9242a2d640 (LWP 2740610) "rocksdb:high"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 766 (Thread 0x7f924342e640 (LWP 2740609) "rocksdb:high"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 765 (Thread 0x7f9243e2f640 (LWP 2740608) "rocksdb:low"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 764 (Thread 0x7f9244830640 (LWP 2740607) "rocksdb:low"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 763 (Thread 0x7f9245231640 (LWP 2740606) "rocksdb:low"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 762 (Thread 0x7f9245c32640 (LWP 2740605) "rocksdb:low"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 761 (Thread 0x7f9246633640 (LWP 2740604) "rocksdb:low"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 760 (Thread 0x7f9247034640 (LWP 2740603) "rocksdb:low"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x00007f95822743bb in rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9582274718 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f958227620f in void* std::__1::__thread_proxy[abi:ue170006] >, void (*)(void*), rocksdb::BGThreadMetadata*> >(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#7 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 759 (Thread 0x7f94ad3ff640 (LWP 2740595) "sst-recovery-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94ad3fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94ad3fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94ad3fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140277123170256, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94ad3cac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94ad3cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94ad3cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 758 (Thread 0x7f94cd7ff640 (LWP 2740592) "region-collecto"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94cd7fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94cd7fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94cd7fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140277106202736, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94cd7cac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94cd7cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94cd7cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 757 (Thread 0x7f95614ff640 (LWP 2740591) "re-metrics"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cfc2f4 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 0x00007f9583cea9cc in std::sys_common::thread_parking::futex::Parker::park_timeout () at library/std/src/sys_common/thread_parking/futex.rs:72 -#3 std::thread::park_timeout () at library/std/src/thread/mod.rs:1133 -#4 0x00007f95806bd330 in std::sync::mpmc::context::Context::wait_until (self=0x7f95614cb2c8, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:130 -#5 0x00007f9580705c4a in std::sync::mpmc::list::{impl#3}::recv::{closure#1}<()> (cx=0x7f95614cb2c8) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:444 -#6 0x00007f95806bd940 in std::sync::mpmc::context::{impl#0}::with::{closure#0}, ()> (cx=, cx=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:50 -#7 std::sync::mpmc::context::{impl#0}::with::{closure#1}, ()> (cell=0x7f95614fba38) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:58 -#8 0x00007f95807089db in std::thread::local::LocalKey>>::try_with>, std::sync::mpmc::context::{impl#0}::with::{closure_env#1}, ()>, ()> (self=0x7f9583e05508, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f95806bd44e in std::sync::mpmc::context::Context::with, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/context.rs:53 -#10 0x00007f9580705b25 in std::sync::mpmc::list::Channel<()>::recv<()> (self=0x7f956a81ea00, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/list.rs:434 -#11 0x00007f95806a480c in std::sync::mpmc::Receiver<()>::recv_deadline<()> (self=0x7f95614cb7a0, deadline=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/mod.rs:340 -#12 0x00007f95806a471a in std::sync::mpmc::Receiver<()>::recv_timeout<()> (self=0x7f95614cb7a0, timeout=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpmc/mod.rs:323 -#13 0x00007f95806e6ea7 in std::sync::mpsc::Receiver<()>::recv_timeout<()> (self=0x7f95614cb7a0, timeout=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sync/mpsc/mod.rs:909 -#14 0x00007f95806547ea in raft_engine::engine::{impl#1}::open_with::{closure#0} () at /home/xzx/.cargo/git/checkouts/raft-engine-35ec7b0b2c07ddd2/de1ec93/src/engine.rs:112 -#15 0x00007f9580690607 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#16 0x00007f95806847fd in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#17 0x00007f958066a9a1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#18 0x00007f958063f2c5 in std::panicking::try::do_call, ()>>, ()> (data=0x7f95614cb900) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#19 0x00007f958064db1b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#20 0x00007f958063ead2 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#21 0x00007f9580684629 in std::panic::catch_unwind, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#22 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#23 0x00007f958065d12f in core::ops::function::FnOnce::call_once, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#24 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#25 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#26 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#27 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#28 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 756 (Thread 0x7f9273635640 (LWP 2740419) "SessionCleaner"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7ffd1e7b6408, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7ffd1e7b6408, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c23628f29 in std::__1::condition_variable::wait_until >, DB::SessionCleaner::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::SessionCleaner::run()::$_0) (this=0x7ffd1e7b6408, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c23628ecf in std::__1::condition_variable::wait_for[abi:ue170006], DB::SessionCleaner::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::SessionCleaner::run()::$_0) (this=0x7ffd1e7b6408, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 -#6 0x0000558c23628e13 in DB::SessionCleaner::run (this=0x7ffd1e7b63d8) at /DATA/disk3/xzx/tiflash/dbms/src/Interpreters/Context.cpp:2131 -#7 0x0000558c1ab89b49 in std::__1::__invoke[abi:ue170006](void (DB::SessionCleaner::*&&)(), DB::SessionCleaner*&&) (__f=@0x7f95702ba9a8: (void (DB::SessionCleaner::*)(class DB::SessionCleaner * const)) 0x558c23628da0 , __a0=@0x7f95702ba9b8: 0x7ffd1e7b63d8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#8 0x0000558c1ab89ace in std::__1::__thread_execute[abi:ue170006] >, void (DB::SessionCleaner::*)(), DB::SessionCleaner*, 2ul>(std::__1::tuple >, void (DB::SessionCleaner::*)(), DB::SessionCleaner*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c1ab898a2 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::SessionCleaner::*)(), DB::SessionCleaner*> >(void*) (__vp=0x7f95702ba9a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 755 (Thread 0x7f9274036640 (LWP 2740418) "civetweb-master"): -#0 0x00007f9570701fdf in poll () from /lib64/libc.so.6 -#1 0x0000558c27835555 in mg_poll (pfd=0x7f940afd42a8, n=1, milliseconds=2000, stop_flag=0x7f95702d7028) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:5936 -#2 0x0000558c2783e895 in master_thread_run (ctx=0x7f95702d7000) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:19236 -#3 0x0000558c278329bc in master_thread (thread_func_param=0x7f95702d7000) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:19347 -#4 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#5 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 754 (Thread 0x7f9274a37640 (LWP 2740417) "civetweb-worker"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x0000558c2783b243 in consume_socket (ctx=0x7f95702d7000, sp=0x7f92b63b52e0, thread_index=1) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:18740 -#3 0x0000558c2783af61 in worker_thread_run (conn=0x7f92b63b49e8) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:18881 -#4 0x0000558c2783295c in worker_thread (thread_func_param=0x7f92b63b49e8) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:19054 -#5 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#6 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 753 (Thread 0x7f9275438640 (LWP 2740416) "civetweb-worker"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x0000558c2783b243 in consume_socket (ctx=0x7f95702d7000, sp=0x7f92b63b48f8, thread_index=0) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:18740 -#3 0x0000558c2783af61 in worker_thread_run (conn=0x7f92b63b4000) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:18881 -#4 0x0000558c2783295c in worker_thread (thread_func_param=0x7f92b63b4000) at /DATA/disk3/xzx/tiflash/contrib/prometheus-cpp/3rdparty/civetweb/src/civetweb.c:19054 -#5 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#6 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 752 (Thread 0x7f9275e39640 (LWP 2740415) "Prometheus"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f95702b0220, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f95702b0220, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c27b7a0f9 in std::__1::condition_variable::wait_until >, Poco::Event::waitImpl(long)::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, Poco::Event::waitImpl(long)::$_0) (this=0x7f95702b0220, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c27b7a06f in std::__1::condition_variable::wait_for[abi:ue170006], Poco::Event::waitImpl(long)::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, Poco::Event::waitImpl(long)::$_0) (this=0x7f95702b0220, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 -#6 0x0000558c27b79e98 in Poco::Event::waitImpl (this=0x7f95702b0220, milliseconds=14999) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Event.cpp:94 -#7 0x0000558c1ac82fed in Poco::Event::tryWait (this=0x7f95702b0220, milliseconds=14999) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/include/Poco/Event.h:109 -#8 0x0000558c27c0173b in Poco::TimedNotificationQueue::wait (this=0x7f95702b0208, interval=14998891) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/TimedNotificationQueue.cpp:165 -#9 0x0000558c27c014ee in Poco::TimedNotificationQueue::waitDequeueNotification (this=0x7f95702b0208) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/TimedNotificationQueue.cpp:99 -#10 0x0000558c27a63e9f in Poco::Util::Timer::run (this=0x7f95702b0200) at /DATA/disk3/xzx/tiflash/contrib/poco/Util/src/Timer.cpp:330 -#11 0x0000558c1ac3fd69 in DB::Timer::run (this=0x7f95702b0200) at /DATA/disk3/xzx/tiflash/dbms/src/Common/Timer.h:32 -#12 0x0000558c27bf81ea in Poco::(anonymous namespace)::RunnableHolder::run (this=0x7f94936ee610) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread.cpp:43 -#13 0x0000558c27bf72ea in Poco::ThreadImpl::runnableEntry (pThread=0x7f95702b02e0) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread_STD.cpp:139 -#14 0x0000558c27bf9760 in std::__1::__invoke[abi:ue170006](void* (*&&)(void*), Poco::ThreadImpl*&&) (__f=@0x7f95702a6a48: 0x558c27bf7280 , __args=@0x7f95702a6a50: 0x7f95702b02e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#15 0x0000558c27bf972e in std::__1::__thread_execute[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*, 2ul>(std::__1::tuple >, void* (*)(void*), Poco::ThreadImpl*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#16 0x0000558c27bf9512 in std::__1::__thread_proxy[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*> >(void*) (__vp=0x7f95702a6a40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#17 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#18 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 751 (Thread 0x7f927683a640 (LWP 2740414) "AsyncMetrics"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7ffd1e7b64b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c23619977 in std::__1::condition_variable::wait_until > > (this=0x7ffd1e7b64b0, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c236119a9 in std::__1::condition_variable::wait_until >, DB::AsynchronousMetrics::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::AsynchronousMetrics::run()::$_0) (this=0x7ffd1e7b64b0, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c23611855 in DB::AsynchronousMetrics::run (this=0x7ffd1e7b6478) at /DATA/disk3/xzx/tiflash/dbms/src/Interpreters/AsynchronousMetrics.cpp:100 -#6 0x0000558c1ab88e38 in DB::AsynchronousMetrics::AsynchronousMetrics(DB::Context&)::{lambda()#1}::operator()() const (this=0x7f94936ee608) at /DATA/disk3/xzx/tiflash/dbms/src/Interpreters/AsynchronousMetrics.h:40 -#7 0x0000558c1ab88df5 in std::__1::__invoke[abi:ue170006](DB::AsynchronousMetrics::AsynchronousMetrics(DB::Context&)::{lambda()#1}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c1ab88d8d in std::__1::__thread_execute[abi:ue170006] >, DB::AsynchronousMetrics::AsynchronousMetrics(DB::Context&)::{lambda()#1}>(std::__1::tuple >, DB::AsynchronousMetrics::AsynchronousMetrics(DB::Context&)::{lambda()#1}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c1ab88942 in std::__1::__thread_proxy[abi:ue170006] >, DB::AsynchronousMetrics::AsynchronousMetrics(DB::Context&)::{lambda()#1}> >(void*) (__vp=0x7f94936ee600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 750 (Thread 0x7f927723b640 (LWP 2740413) "UserCfgReloader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f9570295598, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f9570295598, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c26be9449 in std::__1::condition_variable::wait_until >, DB::ConfigReloader::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::ConfigReloader::run()::$_0) (this=0x7f9570295598, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c26be934f in std::__1::condition_variable::wait_for[abi:ue170006], DB::ConfigReloader::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::ConfigReloader::run()::$_0) (this=0x7f9570295598, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 -#6 0x0000558c26be90d8 in DB::ConfigReloader::run (this=0x7f95702954c0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/Config/ConfigReloader.cpp:67 -#7 0x0000558c26bea279 in std::__1::__invoke[abi:ue170006](void (DB::ConfigReloader::*&&)(), DB::ConfigReloader*&&) (__f=@0x7f95702a69e8: (void (DB::ConfigReloader::*)(class DB::ConfigReloader * const)) 0x558c26be9080 , __a0=@0x7f95702a69f8: 0x7f95702954c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#8 0x0000558c26bea1fe in std::__1::__thread_execute[abi:ue170006] >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*, 2ul>(std::__1::tuple >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c26be9fd2 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*> >(void*) (__vp=0x7f95702a69e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 749 (Thread 0x7f927883c640 (LWP 2740412) "CfgReloader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09dd58, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f956f09dd58, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c26be9449 in std::__1::condition_variable::wait_until >, DB::ConfigReloader::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::ConfigReloader::run()::$_0) (this=0x7f956f09dd58, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c26be934f in std::__1::condition_variable::wait_for[abi:ue170006], DB::ConfigReloader::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::ConfigReloader::run()::$_0) (this=0x7f956f09dd58, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 -#6 0x0000558c26be90d8 in DB::ConfigReloader::run (this=0x7f956f09dc80) at /DATA/disk3/xzx/tiflash/dbms/src/Common/Config/ConfigReloader.cpp:67 -#7 0x0000558c26bea279 in std::__1::__invoke[abi:ue170006](void (DB::ConfigReloader::*&&)(), DB::ConfigReloader*&&) (__f=@0x7f95702b77c8: (void (DB::ConfigReloader::*)(class DB::ConfigReloader * const)) 0x558c26be9080 , __a0=@0x7f95702b77d8: 0x7f956f09dc80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#8 0x0000558c26bea1fe in std::__1::__thread_execute[abi:ue170006] >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*, 2ul>(std::__1::tuple >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c26be9fd2 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::ConfigReloader::*)(), DB::ConfigReloader*> >(void*) (__vp=0x7f95702b77c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 748 (Thread 0x7f927923d640 (LWP 2740411) "TiFlashMain"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c27b79e21 in std::__1::condition_variable::wait(std::__1::unique_lock&, Poco::Event::wait()::$_0) (this=0x7f940b006a40, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c27b79c9b in Poco::Event::wait (this=0x7f940b006a40) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Event.cpp:79 -#5 0x0000558c27bfbbb0 in Poco::PooledThread::run (this=0x7f940b006900) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/ThreadPool.cpp:206 -#6 0x0000558c27bf81ea in Poco::(anonymous namespace)::RunnableHolder::run (this=0x7f94936ee5e0) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread.cpp:43 -#7 0x0000558c27bf72ea in Poco::ThreadImpl::runnableEntry (pThread=0x7f940b006938) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread_STD.cpp:139 -#8 0x0000558c27bf9760 in std::__1::__invoke[abi:ue170006](void* (*&&)(void*), Poco::ThreadImpl*&&) (__f=@0x7f95702b77e8: 0x558c27bf7280 , __args=@0x7f95702b77f0: 0x7f940b006938) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#9 0x0000558c27bf972e in std::__1::__thread_execute[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*, 2ul>(std::__1::tuple >, void* (*)(void*), Poco::ThreadImpl*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#10 0x0000558c27bf9512 in std::__1::__thread_proxy[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*> >(void*) (__vp=0x7f95702b77e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#11 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#12 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 747 (Thread 0x7f927a83e640 (LWP 2740410) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6389668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6389600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6389600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6389600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=399) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ad30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86ad40: 0x7f956f09f800, __args=@0x7f940b86ad48: 399) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ad30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86ad48: 399, __args=@0x7f940b86ad48: 399) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86ad08, args=@0x7f940b86ad48: 399, args=@0x7f940b86ad48: 399) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86ad48: 399, __args=@0x7f940b86ad48: 399) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86ad00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 746 (Thread 0x7f927b23f640 (LWP 2740409) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6389368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6389300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6389300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6389300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=398) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ace0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86acf0: 0x7f956f09f800, __args=@0x7f940b86acf8: 398) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ace0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86acf8: 398, __args=@0x7f940b86acf8: 398) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86acb8, args=@0x7f940b86acf8: 398, args=@0x7f940b86acf8: 398) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86acf8: 398, __args=@0x7f940b86acf8: 398) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86acb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 745 (Thread 0x7f927bc40640 (LWP 2740408) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6389068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6389000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6389000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6389000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=397) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ac90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86aca0: 0x7f956f09f800, __args=@0x7f940b86aca8: 397) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ac90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86aca8: 397, __args=@0x7f940b86aca8: 397) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86ac68, args=@0x7f940b86aca8: 397, args=@0x7f940b86aca8: 397) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86aca8: 397, __args=@0x7f940b86aca8: 397) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86ac60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 744 (Thread 0x7f927c641640 (LWP 2740407) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6308d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6308d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6308d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6308d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=396) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ac40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86ac50: 0x7f956f09f800, __args=@0x7f940b86ac58: 396) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ac40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86ac58: 396, __args=@0x7f940b86ac58: 396) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86ac18, args=@0x7f940b86ac58: 396, args=@0x7f940b86ac58: 396) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86ac58: 396, __args=@0x7f940b86ac58: 396) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86ac10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 743 (Thread 0x7f927d042640 (LWP 2740406) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6308a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6308a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6308a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6308a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=395) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86abf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86ac00: 0x7f956f09f800, __args=@0x7f940b86ac08: 395) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86abf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86ac08: 395, __args=@0x7f940b86ac08: 395) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86abc8, args=@0x7f940b86ac08: 395, args=@0x7f940b86ac08: 395) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86ac08: 395, __args=@0x7f940b86ac08: 395) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86abc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 742 (Thread 0x7f927da43640 (LWP 2740405) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6308768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6308700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6308700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6308700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=394) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aba0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86abb0: 0x7f956f09f800, __args=@0x7f940b86abb8: 394) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aba0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86abb8: 394, __args=@0x7f940b86abb8: 394) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86ab78, args=@0x7f940b86abb8: 394, args=@0x7f940b86abb8: 394) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86abb8: 394, __args=@0x7f940b86abb8: 394) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86ab70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 741 (Thread 0x7f927e444640 (LWP 2740404) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6308468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6308400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6308400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6308400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=393) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ab50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86ab60: 0x7f956f09f800, __args=@0x7f940b86ab68: 393) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ab50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86ab68: 393, __args=@0x7f940b86ab68: 393) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86ab28, args=@0x7f940b86ab68: 393, args=@0x7f940b86ab68: 393) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86ab68: 393, __args=@0x7f940b86ab68: 393) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86ab20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 740 (Thread 0x7f927ee45640 (LWP 2740403) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6308168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6308100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6308100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6308100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=392) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ab00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86ab10: 0x7f956f09f800, __args=@0x7f940b86ab18: 392) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86ab00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86ab18: 392, __args=@0x7f940b86ab18: 392) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86aad8, args=@0x7f940b86ab18: 392, args=@0x7f940b86ab18: 392) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86ab18: 392, __args=@0x7f940b86ab18: 392) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86aad0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 739 (Thread 0x7f927f846640 (LWP 2740402) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6307e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6307e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6307e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6307e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=391) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aab0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86aac0: 0x7f956f09f800, __args=@0x7f940b86aac8: 391) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aab0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86aac8: 391, __args=@0x7f940b86aac8: 391) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86aa88, args=@0x7f940b86aac8: 391, args=@0x7f940b86aac8: 391) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86aac8: 391, __args=@0x7f940b86aac8: 391) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86aa80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 738 (Thread 0x7f9280247640 (LWP 2740401) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6307b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6307b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6307b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6307b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=390) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aa60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86aa70: 0x7f956f09f800, __args=@0x7f940b86aa78: 390) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aa60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86aa78: 390, __args=@0x7f940b86aa78: 390) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86aa38, args=@0x7f940b86aa78: 390, args=@0x7f940b86aa78: 390) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86aa78: 390, __args=@0x7f940b86aa78: 390) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86aa30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 737 (Thread 0x7f9281848640 (LWP 2740400) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6307868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6307800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6307800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6307800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=389) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aa10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86aa20: 0x7f956f09f800, __args=@0x7f940b86aa28: 389) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86aa10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86aa28: 389, __args=@0x7f940b86aa28: 389) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a9e8, args=@0x7f940b86aa28: 389, args=@0x7f940b86aa28: 389) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86aa28: 389, __args=@0x7f940b86aa28: 389) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a9e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 736 (Thread 0x7f9282249640 (LWP 2740399) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6307568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6307500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6307500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6307500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=388) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a9c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a9d0: 0x7f956f09f800, __args=@0x7f940b86a9d8: 388) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a9c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a9d8: 388, __args=@0x7f940b86a9d8: 388) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a998, args=@0x7f940b86a9d8: 388, args=@0x7f940b86a9d8: 388) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a9d8: 388, __args=@0x7f940b86a9d8: 388) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a990) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 735 (Thread 0x7f9282c4a640 (LWP 2740398) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6307268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6307200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6307200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6307200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=387) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a970: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a980: 0x7f956f09f800, __args=@0x7f940b86a988: 387) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a970: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a988: 387, __args=@0x7f940b86a988: 387) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a948, args=@0x7f940b86a988: 387, args=@0x7f940b86a988: 387) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a988: 387, __args=@0x7f940b86a988: 387) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 734 (Thread 0x7f928364b640 (LWP 2740397) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=386) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a920: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a930: 0x7f956f09f800, __args=@0x7f940b86a938: 386) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a920: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a938: 386, __args=@0x7f940b86a938: 386) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a8f8, args=@0x7f940b86a938: 386, args=@0x7f940b86a938: 386) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a938: 386, __args=@0x7f940b86a938: 386) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a8f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 733 (Thread 0x7f928404c640 (LWP 2740396) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=385) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a8d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a8e0: 0x7f956f09f800, __args=@0x7f940b86a8e8: 385) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a8d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a8e8: 385, __args=@0x7f940b86a8e8: 385) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a8a8, args=@0x7f940b86a8e8: 385, args=@0x7f940b86a8e8: 385) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a8e8: 385, __args=@0x7f940b86a8e8: 385) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a8a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 732 (Thread 0x7f9284a4d640 (LWP 2740395) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=384) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a880: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a890: 0x7f956f09f800, __args=@0x7f940b86a898: 384) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a880: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a898: 384, __args=@0x7f940b86a898: 384) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a858, args=@0x7f940b86a898: 384, args=@0x7f940b86a898: 384) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a898: 384, __args=@0x7f940b86a898: 384) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a850) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 731 (Thread 0x7f928544e640 (LWP 2740394) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=383) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a830: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a840: 0x7f956f09f800, __args=@0x7f940b86a848: 383) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a830: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a848: 383, __args=@0x7f940b86a848: 383) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a808, args=@0x7f940b86a848: 383, args=@0x7f940b86a848: 383) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a848: 383, __args=@0x7f940b86a848: 383) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a800) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 730 (Thread 0x7f9285e4f640 (LWP 2740393) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=382) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a7e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a7f0: 0x7f956f09f800, __args=@0x7f940b86a7f8: 382) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a7e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a7f8: 382, __args=@0x7f940b86a7f8: 382) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a7b8, args=@0x7f940b86a7f8: 382, args=@0x7f940b86a7f8: 382) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a7f8: 382, __args=@0x7f940b86a7f8: 382) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a7b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 729 (Thread 0x7f9286850640 (LWP 2740392) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6306068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6306000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6306000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6306000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=381) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a790: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a7a0: 0x7f956f09f800, __args=@0x7f940b86a7a8: 381) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a790: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a7a8: 381, __args=@0x7f940b86a7a8: 381) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a768, args=@0x7f940b86a7a8: 381, args=@0x7f940b86a7a8: 381) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a7a8: 381, __args=@0x7f940b86a7a8: 381) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a760) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 728 (Thread 0x7f9287251640 (LWP 2740391) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6285d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6285d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6285d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6285d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=380) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a740: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a750: 0x7f956f09f800, __args=@0x7f940b86a758: 380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a740: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a758: 380, __args=@0x7f940b86a758: 380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a718, args=@0x7f940b86a758: 380, args=@0x7f940b86a758: 380) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a758: 380, __args=@0x7f940b86a758: 380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a710) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 727 (Thread 0x7f9287c52640 (LWP 2740390) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6285a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6285a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6285a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6285a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=379) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a6f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a700: 0x7f956f09f800, __args=@0x7f940b86a708: 379) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a6f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a708: 379, __args=@0x7f940b86a708: 379) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a6c8, args=@0x7f940b86a708: 379, args=@0x7f940b86a708: 379) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a708: 379, __args=@0x7f940b86a708: 379) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a6c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 726 (Thread 0x7f9288653640 (LWP 2740389) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6285768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6285700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6285700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6285700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=378) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a6a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a6b0: 0x7f956f09f800, __args=@0x7f940b86a6b8: 378) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a6a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a6b8: 378, __args=@0x7f940b86a6b8: 378) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a678, args=@0x7f940b86a6b8: 378, args=@0x7f940b86a6b8: 378) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a6b8: 378, __args=@0x7f940b86a6b8: 378) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a670) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 725 (Thread 0x7f9289054640 (LWP 2740388) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6285468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6285400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6285400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6285400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=377) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a650: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a660: 0x7f956f09f800, __args=@0x7f940b86a668: 377) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a650: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a668: 377, __args=@0x7f940b86a668: 377) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a628, args=@0x7f940b86a668: 377, args=@0x7f940b86a668: 377) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a668: 377, __args=@0x7f940b86a668: 377) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a620) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 724 (Thread 0x7f9289a55640 (LWP 2740387) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6285168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6285100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6285100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6285100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=376) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a600: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a610: 0x7f956f09f800, __args=@0x7f940b86a618: 376) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a600: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a618: 376, __args=@0x7f940b86a618: 376) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a5d8, args=@0x7f940b86a618: 376, args=@0x7f940b86a618: 376) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a618: 376, __args=@0x7f940b86a618: 376) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a5d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 723 (Thread 0x7f928a456640 (LWP 2740386) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6284e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6284e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6284e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6284e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=375) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a5b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a5c0: 0x7f956f09f800, __args=@0x7f940b86a5c8: 375) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a5b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a5c8: 375, __args=@0x7f940b86a5c8: 375) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a588, args=@0x7f940b86a5c8: 375, args=@0x7f940b86a5c8: 375) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a5c8: 375, __args=@0x7f940b86a5c8: 375) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 722 (Thread 0x7f928ae57640 (LWP 2740385) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6284b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6284b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6284b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6284b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=374) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a560: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a570: 0x7f956f09f800, __args=@0x7f940b86a578: 374) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a560: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a578: 374, __args=@0x7f940b86a578: 374) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a538, args=@0x7f940b86a578: 374, args=@0x7f940b86a578: 374) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a578: 374, __args=@0x7f940b86a578: 374) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a530) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 721 (Thread 0x7f928b858640 (LWP 2740384) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6284868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6284800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6284800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6284800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=373) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a510: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a520: 0x7f956f09f800, __args=@0x7f940b86a528: 373) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a510: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a528: 373, __args=@0x7f940b86a528: 373) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a4e8, args=@0x7f940b86a528: 373, args=@0x7f940b86a528: 373) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a528: 373, __args=@0x7f940b86a528: 373) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a4e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 720 (Thread 0x7f928c259640 (LWP 2740383) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6284568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6284500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6284500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6284500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=372) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a4c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a4d0: 0x7f956f09f800, __args=@0x7f940b86a4d8: 372) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a4c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a4d8: 372, __args=@0x7f940b86a4d8: 372) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a498, args=@0x7f940b86a4d8: 372, args=@0x7f940b86a4d8: 372) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a4d8: 372, __args=@0x7f940b86a4d8: 372) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a490) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 719 (Thread 0x7f928cc5a640 (LWP 2740382) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6284268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6284200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6284200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6284200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=371) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a470: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a480: 0x7f956f09f800, __args=@0x7f940b86a488: 371) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a470: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a488: 371, __args=@0x7f940b86a488: 371) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a448, args=@0x7f940b86a488: 371, args=@0x7f940b86a488: 371) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a488: 371, __args=@0x7f940b86a488: 371) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 718 (Thread 0x7f928d65b640 (LWP 2740381) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=370) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a420: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a430: 0x7f956f09f800, __args=@0x7f940b86a438: 370) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a420: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a438: 370, __args=@0x7f940b86a438: 370) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a3f8, args=@0x7f940b86a438: 370, args=@0x7f940b86a438: 370) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a438: 370, __args=@0x7f940b86a438: 370) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a3f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 717 (Thread 0x7f928e05c640 (LWP 2740380) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=369) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a3d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a3e0: 0x7f956f09f800, __args=@0x7f940b86a3e8: 369) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a3d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a3e8: 369, __args=@0x7f940b86a3e8: 369) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a3a8, args=@0x7f940b86a3e8: 369, args=@0x7f940b86a3e8: 369) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a3e8: 369, __args=@0x7f940b86a3e8: 369) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a3a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 716 (Thread 0x7f928ea5d640 (LWP 2740379) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=368) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a380: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a390: 0x7f956f09f800, __args=@0x7f940b86a398: 368) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a380: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a398: 368, __args=@0x7f940b86a398: 368) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a358, args=@0x7f940b86a398: 368, args=@0x7f940b86a398: 368) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a398: 368, __args=@0x7f940b86a398: 368) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 715 (Thread 0x7f928f45e640 (LWP 2740378) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=367) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a330: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a340: 0x7f956f09f800, __args=@0x7f940b86a348: 367) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a330: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a348: 367, __args=@0x7f940b86a348: 367) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a308, args=@0x7f940b86a348: 367, args=@0x7f940b86a348: 367) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a348: 367, __args=@0x7f940b86a348: 367) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 714 (Thread 0x7f928fe5f640 (LWP 2740377) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=366) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a2e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a2f0: 0x7f956f09f800, __args=@0x7f940b86a2f8: 366) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a2e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a2f8: 366, __args=@0x7f940b86a2f8: 366) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a2b8, args=@0x7f940b86a2f8: 366, args=@0x7f940b86a2f8: 366) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a2f8: 366, __args=@0x7f940b86a2f8: 366) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a2b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 713 (Thread 0x7f9290860640 (LWP 2740376) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b6283068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b6283000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b6283000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b6283000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=365) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a290: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a2a0: 0x7f956f09f800, __args=@0x7f940b86a2a8: 365) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a290: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a2a8: 365, __args=@0x7f940b86a2a8: 365) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a268, args=@0x7f940b86a2a8: 365, args=@0x7f940b86a2a8: 365) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a2a8: 365, __args=@0x7f940b86a2a8: 365) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 712 (Thread 0x7f9291261640 (LWP 2740373) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61ffd68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61ffd00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61ffd00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61ffd00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=364) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a240: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a250: 0x7f956f09f800, __args=@0x7f940b86a258: 364) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a240: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a258: 364, __args=@0x7f940b86a258: 364) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a218, args=@0x7f940b86a258: 364, args=@0x7f940b86a258: 364) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a258: 364, __args=@0x7f940b86a258: 364) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a210) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 711 (Thread 0x7f9291c62640 (LWP 2740372) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61ffa68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61ffa00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61ffa00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61ffa00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=363) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a1f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a200: 0x7f956f09f800, __args=@0x7f940b86a208: 363) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a1f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a208: 363, __args=@0x7f940b86a208: 363) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a1c8, args=@0x7f940b86a208: 363, args=@0x7f940b86a208: 363) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a208: 363, __args=@0x7f940b86a208: 363) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a1c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 710 (Thread 0x7f9292663640 (LWP 2740371) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61ff768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61ff700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61ff700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61ff700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=362) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a1a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a1b0: 0x7f956f09f800, __args=@0x7f940b86a1b8: 362) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a1a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a1b8: 362, __args=@0x7f940b86a1b8: 362) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a178, args=@0x7f940b86a1b8: 362, args=@0x7f940b86a1b8: 362) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a1b8: 362, __args=@0x7f940b86a1b8: 362) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a170) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 709 (Thread 0x7f9293064640 (LWP 2740370) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61ff468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61ff400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61ff400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61ff400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=361) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a150: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a160: 0x7f956f09f800, __args=@0x7f940b86a168: 361) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a150: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a168: 361, __args=@0x7f940b86a168: 361) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a128, args=@0x7f940b86a168: 361, args=@0x7f940b86a168: 361) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a168: 361, __args=@0x7f940b86a168: 361) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a120) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 708 (Thread 0x7f9293a65640 (LWP 2740369) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61ff168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61ff100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61ff100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61ff100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=360) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a100: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a110: 0x7f956f09f800, __args=@0x7f940b86a118: 360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a100: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a118: 360, __args=@0x7f940b86a118: 360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a0d8, args=@0x7f940b86a118: 360, args=@0x7f940b86a118: 360) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a118: 360, __args=@0x7f940b86a118: 360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a0d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 707 (Thread 0x7f9294466640 (LWP 2740368) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fee68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fee00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fee00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fee00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=359) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a0b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a0c0: 0x7f956f09f800, __args=@0x7f940b86a0c8: 359) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a0b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a0c8: 359, __args=@0x7f940b86a0c8: 359) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a088, args=@0x7f940b86a0c8: 359, args=@0x7f940b86a0c8: 359) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a0c8: 359, __args=@0x7f940b86a0c8: 359) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 706 (Thread 0x7f9294e67640 (LWP 2740367) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61feb68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61feb00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61feb00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61feb00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=358) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a060: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a070: 0x7f956f09f800, __args=@0x7f940b86a078: 358) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a060: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a078: 358, __args=@0x7f940b86a078: 358) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b86a038, args=@0x7f940b86a078: 358, args=@0x7f940b86a078: 358) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a078: 358, __args=@0x7f940b86a078: 358) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b86a030) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 705 (Thread 0x7f9295868640 (LWP 2740366) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fe868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fe800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fe800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fe800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=357) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a010: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b86a020: 0x7f956f09f800, __args=@0x7f940b86a028: 357) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b86a010: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b86a028: 357, __args=@0x7f940b86a028: 357) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869fe8, args=@0x7f940b86a028: 357, args=@0x7f940b86a028: 357) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b86a028: 357, __args=@0x7f940b86a028: 357) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869fe0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 704 (Thread 0x7f9296269640 (LWP 2740365) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fe568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fe500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fe500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fe500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=356) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869fc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869fd0: 0x7f956f09f800, __args=@0x7f940b869fd8: 356) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869fc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869fd8: 356, __args=@0x7f940b869fd8: 356) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869f98, args=@0x7f940b869fd8: 356, args=@0x7f940b869fd8: 356) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869fd8: 356, __args=@0x7f940b869fd8: 356) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869f90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 703 (Thread 0x7f9296c6a640 (LWP 2740364) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fe268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fe200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fe200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fe200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=355) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869f70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869f80: 0x7f956f09f800, __args=@0x7f940b869f88: 355) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869f70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869f88: 355, __args=@0x7f940b869f88: 355) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869f48, args=@0x7f940b869f88: 355, args=@0x7f940b869f88: 355) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869f88: 355, __args=@0x7f940b869f88: 355) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869f40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 702 (Thread 0x7f929766b640 (LWP 2740363) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fdf68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fdf00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fdf00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fdf00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=354) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869f20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869f30: 0x7f956f09f800, __args=@0x7f940b869f38: 354) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869f20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869f38: 354, __args=@0x7f940b869f38: 354) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869ef8, args=@0x7f940b869f38: 354, args=@0x7f940b869f38: 354) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869f38: 354, __args=@0x7f940b869f38: 354) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869ef0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 701 (Thread 0x7f929806c640 (LWP 2740362) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fdc68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fdc00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fdc00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fdc00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=353) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ed0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869ee0: 0x7f956f09f800, __args=@0x7f940b869ee8: 353) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ed0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869ee8: 353, __args=@0x7f940b869ee8: 353) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869ea8, args=@0x7f940b869ee8: 353, args=@0x7f940b869ee8: 353) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869ee8: 353, __args=@0x7f940b869ee8: 353) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869ea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 700 (Thread 0x7f9298a6d640 (LWP 2740361) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fd968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fd900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fd900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fd900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=352) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869e80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869e90: 0x7f956f09f800, __args=@0x7f940b869e98: 352) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869e80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869e98: 352, __args=@0x7f940b869e98: 352) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869e58, args=@0x7f940b869e98: 352, args=@0x7f940b869e98: 352) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869e98: 352, __args=@0x7f940b869e98: 352) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869e50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 699 (Thread 0x7f929946e640 (LWP 2740360) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fd668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fd600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fd600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fd600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=351) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869e30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869e40: 0x7f956f09f800, __args=@0x7f940b869e48: 351) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869e30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869e48: 351, __args=@0x7f940b869e48: 351) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869e08, args=@0x7f940b869e48: 351, args=@0x7f940b869e48: 351) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869e48: 351, __args=@0x7f940b869e48: 351) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869e00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 698 (Thread 0x7f9299e6f640 (LWP 2740359) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fd368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fd300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fd300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fd300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=350) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869de0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869df0: 0x7f956f09f800, __args=@0x7f940b869df8: 350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869de0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869df8: 350, __args=@0x7f940b869df8: 350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869db8, args=@0x7f940b869df8: 350, args=@0x7f940b869df8: 350) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869df8: 350, __args=@0x7f940b869df8: 350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869db0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 697 (Thread 0x7f929a870640 (LWP 2740358) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f92b61fd068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f92b61fd000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f92b61fd000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f92b61fd000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=349) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869d90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869da0: 0x7f956f09f800, __args=@0x7f940b869da8: 349) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869d90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869da8: 349, __args=@0x7f940b869da8: 349) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869d68, args=@0x7f940b869da8: 349, args=@0x7f940b869da8: 349) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869da8: 349, __args=@0x7f940b869da8: 349) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869d60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 696 (Thread 0x7f929b271640 (LWP 2740357) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0bd68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0bd00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0bd00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0bd00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=348) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869d40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869d50: 0x7f956f09f800, __args=@0x7f940b869d58: 348) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869d40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869d58: 348, __args=@0x7f940b869d58: 348) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869d18, args=@0x7f940b869d58: 348, args=@0x7f940b869d58: 348) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869d58: 348, __args=@0x7f940b869d58: 348) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869d10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 695 (Thread 0x7f929bc72640 (LWP 2740356) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0ba68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0ba00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0ba00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0ba00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=347) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869cf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869d00: 0x7f956f09f800, __args=@0x7f940b869d08: 347) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869cf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869d08: 347, __args=@0x7f940b869d08: 347) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869cc8, args=@0x7f940b869d08: 347, args=@0x7f940b869d08: 347) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869d08: 347, __args=@0x7f940b869d08: 347) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 694 (Thread 0x7f929c673640 (LWP 2740355) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0b768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0b700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0b700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0b700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=346) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ca0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869cb0: 0x7f956f09f800, __args=@0x7f940b869cb8: 346) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ca0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869cb8: 346, __args=@0x7f940b869cb8: 346) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869c78, args=@0x7f940b869cb8: 346, args=@0x7f940b869cb8: 346) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869cb8: 346, __args=@0x7f940b869cb8: 346) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869c70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 693 (Thread 0x7f929d074640 (LWP 2740354) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0b468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0b400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0b400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0b400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=345) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869c50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869c60: 0x7f956f09f800, __args=@0x7f940b869c68: 345) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869c50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869c68: 345, __args=@0x7f940b869c68: 345) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869c28, args=@0x7f940b869c68: 345, args=@0x7f940b869c68: 345) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869c68: 345, __args=@0x7f940b869c68: 345) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869c20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 692 (Thread 0x7f929da75640 (LWP 2740353) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0b168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0b100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0b100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0b100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=344) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869c00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869c10: 0x7f956f09f800, __args=@0x7f940b869c18: 344) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869c00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869c18: 344, __args=@0x7f940b869c18: 344) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869bd8, args=@0x7f940b869c18: 344, args=@0x7f940b869c18: 344) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869c18: 344, __args=@0x7f940b869c18: 344) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869bd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 691 (Thread 0x7f929e476640 (LWP 2740352) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0ae68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0ae00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0ae00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0ae00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=343) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869bb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869bc0: 0x7f956f09f800, __args=@0x7f940b869bc8: 343) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869bb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869bc8: 343, __args=@0x7f940b869bc8: 343) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869b88, args=@0x7f940b869bc8: 343, args=@0x7f940b869bc8: 343) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869bc8: 343, __args=@0x7f940b869bc8: 343) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869b80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 690 (Thread 0x7f929ee77640 (LWP 2740351) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0ab68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0ab00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0ab00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0ab00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=342) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869b60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869b70: 0x7f956f09f800, __args=@0x7f940b869b78: 342) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869b60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869b78: 342, __args=@0x7f940b869b78: 342) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869b38, args=@0x7f940b869b78: 342, args=@0x7f940b869b78: 342) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869b78: 342, __args=@0x7f940b869b78: 342) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869b30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 689 (Thread 0x7f929f878640 (LWP 2740350) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0a868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0a800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0a800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0a800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=341) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869b10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869b20: 0x7f956f09f800, __args=@0x7f940b869b28: 341) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869b10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869b28: 341, __args=@0x7f940b869b28: 341) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869ae8, args=@0x7f940b869b28: 341, args=@0x7f940b869b28: 341) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869b28: 341, __args=@0x7f940b869b28: 341) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869ae0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 688 (Thread 0x7f92a0279640 (LWP 2740349) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0a568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0a500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0a500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0a500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=340) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ac0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869ad0: 0x7f956f09f800, __args=@0x7f940b869ad8: 340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869ac0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869ad8: 340, __args=@0x7f940b869ad8: 340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869a98, args=@0x7f940b869ad8: 340, args=@0x7f940b869ad8: 340) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869ad8: 340, __args=@0x7f940b869ad8: 340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869a90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 687 (Thread 0x7f92a0c7a640 (LWP 2740348) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba0a268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba0a200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba0a200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba0a200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=339) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869a70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869a80: 0x7f956f09f800, __args=@0x7f940b869a88: 339) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869a70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869a88: 339, __args=@0x7f940b869a88: 339) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869a48, args=@0x7f940b869a88: 339, args=@0x7f940b869a88: 339) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869a88: 339, __args=@0x7f940b869a88: 339) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869a40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 686 (Thread 0x7f92a167b640 (LWP 2740347) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=338) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869a20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869a30: 0x7f956f09f800, __args=@0x7f940b869a38: 338) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869a20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869a38: 338, __args=@0x7f940b869a38: 338) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8699f8, args=@0x7f940b869a38: 338, args=@0x7f940b869a38: 338) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869a38: 338, __args=@0x7f940b869a38: 338) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8699f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 685 (Thread 0x7f92a207c640 (LWP 2740346) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=337) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8699d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8699e0: 0x7f956f09f800, __args=@0x7f940b8699e8: 337) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8699d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8699e8: 337, __args=@0x7f940b8699e8: 337) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8699a8, args=@0x7f940b8699e8: 337, args=@0x7f940b8699e8: 337) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8699e8: 337, __args=@0x7f940b8699e8: 337) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8699a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 684 (Thread 0x7f92a2a7d640 (LWP 2740345) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=336) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869980: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869990: 0x7f956f09f800, __args=@0x7f940b869998: 336) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869980: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869998: 336, __args=@0x7f940b869998: 336) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869958, args=@0x7f940b869998: 336, args=@0x7f940b869998: 336) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869998: 336, __args=@0x7f940b869998: 336) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869950) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 683 (Thread 0x7f92a347e640 (LWP 2740344) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=335) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869930: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869940: 0x7f956f09f800, __args=@0x7f940b869948: 335) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869930: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869948: 335, __args=@0x7f940b869948: 335) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869908, args=@0x7f940b869948: 335, args=@0x7f940b869948: 335) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869948: 335, __args=@0x7f940b869948: 335) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 682 (Thread 0x7f92a3e7f640 (LWP 2740343) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=334) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8698e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8698f0: 0x7f956f09f800, __args=@0x7f940b8698f8: 334) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8698e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8698f8: 334, __args=@0x7f940b8698f8: 334) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8698b8, args=@0x7f940b8698f8: 334, args=@0x7f940b8698f8: 334) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8698f8: 334, __args=@0x7f940b8698f8: 334) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8698b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 681 (Thread 0x7f92a4880640 (LWP 2740342) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba09068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba09000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba09000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba09000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=333) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869890: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8698a0: 0x7f956f09f800, __args=@0x7f940b8698a8: 333) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869890: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8698a8: 333, __args=@0x7f940b8698a8: 333) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869868, args=@0x7f940b8698a8: 333, args=@0x7f940b8698a8: 333) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8698a8: 333, __args=@0x7f940b8698a8: 333) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869860) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 680 (Thread 0x7f92a5281640 (LWP 2740341) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba08d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba08d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba08d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba08d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=332) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869840: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869850: 0x7f956f09f800, __args=@0x7f940b869858: 332) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869840: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869858: 332, __args=@0x7f940b869858: 332) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869818, args=@0x7f940b869858: 332, args=@0x7f940b869858: 332) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869858: 332, __args=@0x7f940b869858: 332) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869810) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 679 (Thread 0x7f92a5c82640 (LWP 2740340) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba08a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba08a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba08a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba08a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=331) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8697f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869800: 0x7f956f09f800, __args=@0x7f940b869808: 331) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8697f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869808: 331, __args=@0x7f940b869808: 331) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8697c8, args=@0x7f940b869808: 331, args=@0x7f940b869808: 331) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869808: 331, __args=@0x7f940b869808: 331) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8697c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 678 (Thread 0x7f92a6683640 (LWP 2740339) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba08768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba08700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba08700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba08700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=330) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8697a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8697b0: 0x7f956f09f800, __args=@0x7f940b8697b8: 330) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8697a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8697b8: 330, __args=@0x7f940b8697b8: 330) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869778, args=@0x7f940b8697b8: 330, args=@0x7f940b8697b8: 330) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8697b8: 330, __args=@0x7f940b8697b8: 330) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869770) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 677 (Thread 0x7f92a7084640 (LWP 2740338) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba08468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba08400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba08400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba08400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=329) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869750: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869760: 0x7f956f09f800, __args=@0x7f940b869768: 329) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869750: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869768: 329, __args=@0x7f940b869768: 329) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869728, args=@0x7f940b869768: 329, args=@0x7f940b869768: 329) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869768: 329, __args=@0x7f940b869768: 329) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 676 (Thread 0x7f92a7a85640 (LWP 2740337) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba08168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba08100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba08100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba08100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=328) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869700: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869710: 0x7f956f09f800, __args=@0x7f940b869718: 328) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869700: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869718: 328, __args=@0x7f940b869718: 328) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8696d8, args=@0x7f940b869718: 328, args=@0x7f940b869718: 328) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869718: 328, __args=@0x7f940b869718: 328) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8696d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 675 (Thread 0x7f92a8486640 (LWP 2740336) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba07e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba07e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba07e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba07e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=327) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8696b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8696c0: 0x7f956f09f800, __args=@0x7f940b8696c8: 327) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8696b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8696c8: 327, __args=@0x7f940b8696c8: 327) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869688, args=@0x7f940b8696c8: 327, args=@0x7f940b8696c8: 327) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8696c8: 327, __args=@0x7f940b8696c8: 327) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 674 (Thread 0x7f92a8e87640 (LWP 2740335) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba07b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba07b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba07b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba07b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=326) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869660: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869670: 0x7f956f09f800, __args=@0x7f940b869678: 326) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869660: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869678: 326, __args=@0x7f940b869678: 326) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869638, args=@0x7f940b869678: 326, args=@0x7f940b869678: 326) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869678: 326, __args=@0x7f940b869678: 326) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869630) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 673 (Thread 0x7f92a9888640 (LWP 2740334) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba07868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba07800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba07800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba07800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=325) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869610: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869620: 0x7f956f09f800, __args=@0x7f940b869628: 325) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869610: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869628: 325, __args=@0x7f940b869628: 325) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8695e8, args=@0x7f940b869628: 325, args=@0x7f940b869628: 325) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869628: 325, __args=@0x7f940b869628: 325) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8695e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 672 (Thread 0x7f92aa289640 (LWP 2740333) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba07568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba07500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba07500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba07500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=324) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8695c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8695d0: 0x7f956f09f800, __args=@0x7f940b8695d8: 324) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8695c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8695d8: 324, __args=@0x7f940b8695d8: 324) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869598, args=@0x7f940b8695d8: 324, args=@0x7f940b8695d8: 324) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8695d8: 324, __args=@0x7f940b8695d8: 324) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869590) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 671 (Thread 0x7f92aac8a640 (LWP 2740332) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba07268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba07200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba07200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba07200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=323) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869570: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869580: 0x7f956f09f800, __args=@0x7f940b869588: 323) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869570: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869588: 323, __args=@0x7f940b869588: 323) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869548, args=@0x7f940b869588: 323, args=@0x7f940b869588: 323) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869588: 323, __args=@0x7f940b869588: 323) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 670 (Thread 0x7f92ab68b640 (LWP 2740331) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=322) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869520: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869530: 0x7f956f09f800, __args=@0x7f940b869538: 322) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869520: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869538: 322, __args=@0x7f940b869538: 322) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8694f8, args=@0x7f940b869538: 322, args=@0x7f940b869538: 322) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869538: 322, __args=@0x7f940b869538: 322) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8694f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 669 (Thread 0x7f92ac08c640 (LWP 2740330) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=321) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8694d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8694e0: 0x7f956f09f800, __args=@0x7f940b8694e8: 321) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8694d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8694e8: 321, __args=@0x7f940b8694e8: 321) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8694a8, args=@0x7f940b8694e8: 321, args=@0x7f940b8694e8: 321) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8694e8: 321, __args=@0x7f940b8694e8: 321) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8694a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 668 (Thread 0x7f92aca8d640 (LWP 2740329) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=320) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869480: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869490: 0x7f956f09f800, __args=@0x7f940b869498: 320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869480: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869498: 320, __args=@0x7f940b869498: 320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869458, args=@0x7f940b869498: 320, args=@0x7f940b869498: 320) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869498: 320, __args=@0x7f940b869498: 320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869450) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 667 (Thread 0x7f92ad48e640 (LWP 2740328) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=319) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869430: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869440: 0x7f956f09f800, __args=@0x7f940b869448: 319) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869430: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869448: 319, __args=@0x7f940b869448: 319) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869408, args=@0x7f940b869448: 319, args=@0x7f940b869448: 319) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869448: 319, __args=@0x7f940b869448: 319) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 666 (Thread 0x7f92ade8f640 (LWP 2740327) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=318) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8693e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8693f0: 0x7f956f09f800, __args=@0x7f940b8693f8: 318) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8693e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8693f8: 318, __args=@0x7f940b8693f8: 318) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8693b8, args=@0x7f940b8693f8: 318, args=@0x7f940b8693f8: 318) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8693f8: 318, __args=@0x7f940b8693f8: 318) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8693b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 665 (Thread 0x7f92ae890640 (LWP 2740326) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940ba06068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940ba06000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940ba06000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940ba06000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=317) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869390: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8693a0: 0x7f956f09f800, __args=@0x7f940b8693a8: 317) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869390: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8693a8: 317, __args=@0x7f940b8693a8: 317) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869368, args=@0x7f940b8693a8: 317, args=@0x7f940b8693a8: 317) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8693a8: 317, __args=@0x7f940b8693a8: 317) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 664 (Thread 0x7f92af291640 (LWP 2740325) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e5d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e5d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e5d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e5d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=316) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869340: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869350: 0x7f956f09f800, __args=@0x7f940b869358: 316) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869340: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869358: 316, __args=@0x7f940b869358: 316) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869318, args=@0x7f940b869358: 316, args=@0x7f940b869358: 316) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869358: 316, __args=@0x7f940b869358: 316) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869310) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 663 (Thread 0x7f92afc92640 (LWP 2740324) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e5a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e5a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e5a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e5a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=315) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8692f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869300: 0x7f956f09f800, __args=@0x7f940b869308: 315) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8692f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869308: 315, __args=@0x7f940b869308: 315) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8692c8, args=@0x7f940b869308: 315, args=@0x7f940b869308: 315) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869308: 315, __args=@0x7f940b869308: 315) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8692c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 662 (Thread 0x7f92b0693640 (LWP 2740323) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e5768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e5700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e5700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e5700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=314) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8692a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8692b0: 0x7f956f09f800, __args=@0x7f940b8692b8: 314) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8692a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8692b8: 314, __args=@0x7f940b8692b8: 314) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869278, args=@0x7f940b8692b8: 314, args=@0x7f940b8692b8: 314) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8692b8: 314, __args=@0x7f940b8692b8: 314) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869270) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 661 (Thread 0x7f92b1094640 (LWP 2740322) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e5468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e5400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e5400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e5400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=313) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869250: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869260: 0x7f956f09f800, __args=@0x7f940b869268: 313) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869250: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869268: 313, __args=@0x7f940b869268: 313) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869228, args=@0x7f940b869268: 313, args=@0x7f940b869268: 313) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869268: 313, __args=@0x7f940b869268: 313) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 660 (Thread 0x7f92b1a95640 (LWP 2740321) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e5168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e5100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e5100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e5100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=312) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869200: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869210: 0x7f956f09f800, __args=@0x7f940b869218: 312) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869200: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869218: 312, __args=@0x7f940b869218: 312) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8691d8, args=@0x7f940b869218: 312, args=@0x7f940b869218: 312) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869218: 312, __args=@0x7f940b869218: 312) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8691d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 659 (Thread 0x7f92b2496640 (LWP 2740320) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e4e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e4e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e4e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e4e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=311) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8691b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8691c0: 0x7f956f09f800, __args=@0x7f940b8691c8: 311) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8691b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8691c8: 311, __args=@0x7f940b8691c8: 311) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869188, args=@0x7f940b8691c8: 311, args=@0x7f940b8691c8: 311) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8691c8: 311, __args=@0x7f940b8691c8: 311) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 658 (Thread 0x7f92b2e97640 (LWP 2740319) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e4b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e4b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e4b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e4b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=310) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869160: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869170: 0x7f956f09f800, __args=@0x7f940b869178: 310) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869160: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869178: 310, __args=@0x7f940b869178: 310) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869138, args=@0x7f940b869178: 310, args=@0x7f940b869178: 310) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869178: 310, __args=@0x7f940b869178: 310) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869130) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 657 (Thread 0x7f92b3898640 (LWP 2740318) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e4868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e4800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e4800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e4800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=309) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869110: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869120: 0x7f956f09f800, __args=@0x7f940b869128: 309) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869110: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869128: 309, __args=@0x7f940b869128: 309) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8690e8, args=@0x7f940b869128: 309, args=@0x7f940b869128: 309) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869128: 309, __args=@0x7f940b869128: 309) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8690e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 656 (Thread 0x7f92b4299640 (LWP 2740317) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e4568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e4500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e4500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e4500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=308) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8690c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8690d0: 0x7f956f09f800, __args=@0x7f940b8690d8: 308) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8690c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8690d8: 308, __args=@0x7f940b8690d8: 308) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869098, args=@0x7f940b8690d8: 308, args=@0x7f940b8690d8: 308) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8690d8: 308, __args=@0x7f940b8690d8: 308) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869090) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 655 (Thread 0x7f92b4c9a640 (LWP 2740316) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e4268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e4200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e4200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e4200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=307) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869070: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869080: 0x7f956f09f800, __args=@0x7f940b869088: 307) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869070: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869088: 307, __args=@0x7f940b869088: 307) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b869048, args=@0x7f940b869088: 307, args=@0x7f940b869088: 307) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869088: 307, __args=@0x7f940b869088: 307) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b869040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 654 (Thread 0x7f92b569b640 (LWP 2740315) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=306) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869020: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b869030: 0x7f956f09f800, __args=@0x7f940b869038: 306) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b869020: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b869038: 306, __args=@0x7f940b869038: 306) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868ff8, args=@0x7f940b869038: 306, args=@0x7f940b869038: 306) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b869038: 306, __args=@0x7f940b869038: 306) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868ff0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 653 (Thread 0x7f92b609c640 (LWP 2740314) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=305) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868fd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868fe0: 0x7f956f09f800, __args=@0x7f940b868fe8: 305) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868fd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868fe8: 305, __args=@0x7f940b868fe8: 305) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868fa8, args=@0x7f940b868fe8: 305, args=@0x7f940b868fe8: 305) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868fe8: 305, __args=@0x7f940b868fe8: 305) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868fa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 652 (Thread 0x7f92b7a9d640 (LWP 2740313) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=304) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868f80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868f90: 0x7f956f09f800, __args=@0x7f940b868f98: 304) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868f80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868f98: 304, __args=@0x7f940b868f98: 304) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868f58, args=@0x7f940b868f98: 304, args=@0x7f940b868f98: 304) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868f98: 304, __args=@0x7f940b868f98: 304) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868f50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 651 (Thread 0x7f92b849e640 (LWP 2740312) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=303) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868f30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868f40: 0x7f956f09f800, __args=@0x7f940b868f48: 303) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868f30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868f48: 303, __args=@0x7f940b868f48: 303) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868f08, args=@0x7f940b868f48: 303, args=@0x7f940b868f48: 303) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868f48: 303, __args=@0x7f940b868f48: 303) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868f00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 650 (Thread 0x7f92b8e9f640 (LWP 2740311) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=302) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868ee0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868ef0: 0x7f956f09f800, __args=@0x7f940b868ef8: 302) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868ee0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868ef8: 302, __args=@0x7f940b868ef8: 302) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868eb8, args=@0x7f940b868ef8: 302, args=@0x7f940b868ef8: 302) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868ef8: 302, __args=@0x7f940b868ef8: 302) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868eb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 649 (Thread 0x7f92b98a0640 (LWP 2740310) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b9e3068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b9e3000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b9e3000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b9e3000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=301) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868e90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868ea0: 0x7f956f09f800, __args=@0x7f940b868ea8: 301) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868e90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868ea8: 301, __args=@0x7f940b868ea8: 301) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868e68, args=@0x7f940b868ea8: 301, args=@0x7f940b868ea8: 301) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868ea8: 301, __args=@0x7f940b868ea8: 301) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868e60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 648 (Thread 0x7f92ba2a1640 (LWP 2740309) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b962d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b962d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b962d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b962d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=300) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868e40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868e50: 0x7f956f09f800, __args=@0x7f940b868e58: 300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868e40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868e58: 300, __args=@0x7f940b868e58: 300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868e18, args=@0x7f940b868e58: 300, args=@0x7f940b868e58: 300) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868e58: 300, __args=@0x7f940b868e58: 300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868e10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 647 (Thread 0x7f92baca2640 (LWP 2740308) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b962a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b962a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b962a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b962a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=299) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868df0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868e00: 0x7f956f09f800, __args=@0x7f940b868e08: 299) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868df0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868e08: 299, __args=@0x7f940b868e08: 299) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868dc8, args=@0x7f940b868e08: 299, args=@0x7f940b868e08: 299) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868e08: 299, __args=@0x7f940b868e08: 299) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868dc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 646 (Thread 0x7f92bb6a3640 (LWP 2740307) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b962768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b962700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b962700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b962700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=298) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868da0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868db0: 0x7f956f09f800, __args=@0x7f940b868db8: 298) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868da0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868db8: 298, __args=@0x7f940b868db8: 298) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868d78, args=@0x7f940b868db8: 298, args=@0x7f940b868db8: 298) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868db8: 298, __args=@0x7f940b868db8: 298) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868d70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 645 (Thread 0x7f92bc0a4640 (LWP 2740306) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b962468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b962400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b962400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b962400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=297) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868d50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868d60: 0x7f956f09f800, __args=@0x7f940b868d68: 297) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868d50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868d68: 297, __args=@0x7f940b868d68: 297) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868d28, args=@0x7f940b868d68: 297, args=@0x7f940b868d68: 297) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868d68: 297, __args=@0x7f940b868d68: 297) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868d20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 644 (Thread 0x7f92bd4a5640 (LWP 2740305) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b962168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b962100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b962100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b962100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=296) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868d00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868d10: 0x7f956f09f800, __args=@0x7f940b868d18: 296) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868d00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868d18: 296, __args=@0x7f940b868d18: 296) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868cd8, args=@0x7f940b868d18: 296, args=@0x7f940b868d18: 296) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868d18: 296, __args=@0x7f940b868d18: 296) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868cd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 643 (Thread 0x7f92bdea6640 (LWP 2740304) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b961e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b961e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b961e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b961e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=295) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868cb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868cc0: 0x7f956f09f800, __args=@0x7f940b868cc8: 295) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868cb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868cc8: 295, __args=@0x7f940b868cc8: 295) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868c88, args=@0x7f940b868cc8: 295, args=@0x7f940b868cc8: 295) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868cc8: 295, __args=@0x7f940b868cc8: 295) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868c80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 642 (Thread 0x7f92be8a7640 (LWP 2740303) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b961b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b961b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b961b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b961b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=294) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868c60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868c70: 0x7f956f09f800, __args=@0x7f940b868c78: 294) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868c60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868c78: 294, __args=@0x7f940b868c78: 294) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868c38, args=@0x7f940b868c78: 294, args=@0x7f940b868c78: 294) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868c78: 294, __args=@0x7f940b868c78: 294) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868c30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 641 (Thread 0x7f92bf2a8640 (LWP 2740302) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b961868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b961800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b961800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b961800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=293) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868c10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868c20: 0x7f956f09f800, __args=@0x7f940b868c28: 293) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868c10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868c28: 293, __args=@0x7f940b868c28: 293) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868be8, args=@0x7f940b868c28: 293, args=@0x7f940b868c28: 293) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868c28: 293, __args=@0x7f940b868c28: 293) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868be0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 640 (Thread 0x7f92bfca9640 (LWP 2740301) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b961568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b961500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b961500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b961500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=292) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868bc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868bd0: 0x7f956f09f800, __args=@0x7f940b868bd8: 292) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868bc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868bd8: 292, __args=@0x7f940b868bd8: 292) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868b98, args=@0x7f940b868bd8: 292, args=@0x7f940b868bd8: 292) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868bd8: 292, __args=@0x7f940b868bd8: 292) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868b90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 639 (Thread 0x7f92c06aa640 (LWP 2740300) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b961268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b961200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b961200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b961200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=291) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868b70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868b80: 0x7f956f09f800, __args=@0x7f940b868b88: 291) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868b70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868b88: 291, __args=@0x7f940b868b88: 291) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868b48, args=@0x7f940b868b88: 291, args=@0x7f940b868b88: 291) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868b88: 291, __args=@0x7f940b868b88: 291) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 638 (Thread 0x7f92c10ab640 (LWP 2740299) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b960f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=290) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868b20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868b30: 0x7f956f09f800, __args=@0x7f940b868b38: 290) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868b20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868b38: 290, __args=@0x7f940b868b38: 290) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868af8, args=@0x7f940b868b38: 290, args=@0x7f940b868b38: 290) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868b38: 290, __args=@0x7f940b868b38: 290) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868af0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 637 (Thread 0x7f92c1aac640 (LWP 2740298) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b960c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=289) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868ad0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868ae0: 0x7f956f09f800, __args=@0x7f940b868ae8: 289) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868ad0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868ae8: 289, __args=@0x7f940b868ae8: 289) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868aa8, args=@0x7f940b868ae8: 289, args=@0x7f940b868ae8: 289) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868ae8: 289, __args=@0x7f940b868ae8: 289) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868aa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 636 (Thread 0x7f92c24ad640 (LWP 2740297) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b960900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=288) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868a80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868a90: 0x7f956f09f800, __args=@0x7f940b868a98: 288) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868a80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868a98: 288, __args=@0x7f940b868a98: 288) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868a58, args=@0x7f940b868a98: 288, args=@0x7f940b868a98: 288) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868a98: 288, __args=@0x7f940b868a98: 288) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868a50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 635 (Thread 0x7f92c2eae640 (LWP 2740296) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b960600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=287) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868a30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868a40: 0x7f956f09f800, __args=@0x7f940b868a48: 287) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868a30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868a48: 287, __args=@0x7f940b868a48: 287) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868a08, args=@0x7f940b868a48: 287, args=@0x7f940b868a48: 287) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868a48: 287, __args=@0x7f940b868a48: 287) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 634 (Thread 0x7f92c38af640 (LWP 2740295) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b960300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=286) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8689e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8689f0: 0x7f956f09f800, __args=@0x7f940b8689f8: 286) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8689e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8689f8: 286, __args=@0x7f940b8689f8: 286) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8689b8, args=@0x7f940b8689f8: 286, args=@0x7f940b8689f8: 286) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8689f8: 286, __args=@0x7f940b8689f8: 286) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8689b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 633 (Thread 0x7f92c42b0640 (LWP 2740294) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b960068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b960000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b960000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b960000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=285) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868990: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8689a0: 0x7f956f09f800, __args=@0x7f940b8689a8: 285) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868990: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8689a8: 285, __args=@0x7f940b8689a8: 285) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868968, args=@0x7f940b8689a8: 285, args=@0x7f940b8689a8: 285) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8689a8: 285, __args=@0x7f940b8689a8: 285) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868960) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 632 (Thread 0x7f92c4cb1640 (LWP 2740293) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dfd68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dfd00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dfd00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dfd00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=284) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868940: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868950: 0x7f956f09f800, __args=@0x7f940b868958: 284) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868940: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868958: 284, __args=@0x7f940b868958: 284) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868918, args=@0x7f940b868958: 284, args=@0x7f940b868958: 284) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868958: 284, __args=@0x7f940b868958: 284) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868910) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 631 (Thread 0x7f92c56b2640 (LWP 2740292) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dfa68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dfa00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dfa00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dfa00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=283) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8688f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868900: 0x7f956f09f800, __args=@0x7f940b868908: 283) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8688f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868908: 283, __args=@0x7f940b868908: 283) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8688c8, args=@0x7f940b868908: 283, args=@0x7f940b868908: 283) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868908: 283, __args=@0x7f940b868908: 283) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8688c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 630 (Thread 0x7f92c60b3640 (LWP 2740291) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8df768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8df700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8df700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8df700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=282) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8688a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8688b0: 0x7f956f09f800, __args=@0x7f940b8688b8: 282) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8688a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8688b8: 282, __args=@0x7f940b8688b8: 282) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868878, args=@0x7f940b8688b8: 282, args=@0x7f940b8688b8: 282) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8688b8: 282, __args=@0x7f940b8688b8: 282) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868870) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 629 (Thread 0x7f92c6ab4640 (LWP 2740290) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8df468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8df400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8df400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8df400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=281) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868850: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868860: 0x7f956f09f800, __args=@0x7f940b868868: 281) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868850: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868868: 281, __args=@0x7f940b868868: 281) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868828, args=@0x7f940b868868: 281, args=@0x7f940b868868: 281) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868868: 281, __args=@0x7f940b868868: 281) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868820) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 628 (Thread 0x7f92c74b5640 (LWP 2740289) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8df168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8df100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8df100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8df100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=280) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868800: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868810: 0x7f956f09f800, __args=@0x7f940b868818: 280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868800: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868818: 280, __args=@0x7f940b868818: 280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8687d8, args=@0x7f940b868818: 280, args=@0x7f940b868818: 280) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868818: 280, __args=@0x7f940b868818: 280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8687d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 627 (Thread 0x7f92c7eb6640 (LWP 2740288) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dee68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dee00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dee00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dee00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=279) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8687b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8687c0: 0x7f956f09f800, __args=@0x7f940b8687c8: 279) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8687b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8687c8: 279, __args=@0x7f940b8687c8: 279) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868788, args=@0x7f940b8687c8: 279, args=@0x7f940b8687c8: 279) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8687c8: 279, __args=@0x7f940b8687c8: 279) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 626 (Thread 0x7f92c8fb7640 (LWP 2740287) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8deb68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8deb00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8deb00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8deb00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=278) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868760: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868770: 0x7f956f09f800, __args=@0x7f940b868778: 278) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868760: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868778: 278, __args=@0x7f940b868778: 278) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868738, args=@0x7f940b868778: 278, args=@0x7f940b868778: 278) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868778: 278, __args=@0x7f940b868778: 278) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868730) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 625 (Thread 0x7f92c99b8640 (LWP 2740286) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8de868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8de800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8de800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8de800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=277) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868710: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868720: 0x7f956f09f800, __args=@0x7f940b868728: 277) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868710: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868728: 277, __args=@0x7f940b868728: 277) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8686e8, args=@0x7f940b868728: 277, args=@0x7f940b868728: 277) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868728: 277, __args=@0x7f940b868728: 277) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8686e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 624 (Thread 0x7f92ca3b9640 (LWP 2740285) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8de568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8de500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8de500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8de500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=276) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8686c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8686d0: 0x7f956f09f800, __args=@0x7f940b8686d8: 276) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8686c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8686d8: 276, __args=@0x7f940b8686d8: 276) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868698, args=@0x7f940b8686d8: 276, args=@0x7f940b8686d8: 276) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8686d8: 276, __args=@0x7f940b8686d8: 276) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868690) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 623 (Thread 0x7f92cbdba640 (LWP 2740284) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8de268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8de200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8de200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8de200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=275) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868670: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868680: 0x7f956f09f800, __args=@0x7f940b868688: 275) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868670: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868688: 275, __args=@0x7f940b868688: 275) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868648, args=@0x7f940b868688: 275, args=@0x7f940b868688: 275) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868688: 275, __args=@0x7f940b868688: 275) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 622 (Thread 0x7f92cc7bb640 (LWP 2740283) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8ddf68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8ddf00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8ddf00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8ddf00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=274) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868620: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868630: 0x7f956f09f800, __args=@0x7f940b868638: 274) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868620: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868638: 274, __args=@0x7f940b868638: 274) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8685f8, args=@0x7f940b868638: 274, args=@0x7f940b868638: 274) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868638: 274, __args=@0x7f940b868638: 274) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8685f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 621 (Thread 0x7f92cd1bc640 (LWP 2740282) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8ddc68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8ddc00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8ddc00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8ddc00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=273) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8685d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8685e0: 0x7f956f09f800, __args=@0x7f940b8685e8: 273) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8685d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8685e8: 273, __args=@0x7f940b8685e8: 273) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8685a8, args=@0x7f940b8685e8: 273, args=@0x7f940b8685e8: 273) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8685e8: 273, __args=@0x7f940b8685e8: 273) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8685a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 620 (Thread 0x7f92cdbbd640 (LWP 2740281) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dd968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dd900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dd900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dd900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=272) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868580: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868590: 0x7f956f09f800, __args=@0x7f940b868598: 272) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868580: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868598: 272, __args=@0x7f940b868598: 272) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868558, args=@0x7f940b868598: 272, args=@0x7f940b868598: 272) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868598: 272, __args=@0x7f940b868598: 272) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868550) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 619 (Thread 0x7f92ce5be640 (LWP 2740280) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dd668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dd600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dd600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dd600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=271) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868530: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868540: 0x7f956f09f800, __args=@0x7f940b868548: 271) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868530: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868548: 271, __args=@0x7f940b868548: 271) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868508, args=@0x7f940b868548: 271, args=@0x7f940b868548: 271) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868548: 271, __args=@0x7f940b868548: 271) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 618 (Thread 0x7f92cefbf640 (LWP 2740279) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dd368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dd300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dd300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dd300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=270) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8684e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8684f0: 0x7f956f09f800, __args=@0x7f940b8684f8: 270) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8684e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8684f8: 270, __args=@0x7f940b8684f8: 270) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8684b8, args=@0x7f940b8684f8: 270, args=@0x7f940b8684f8: 270) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8684f8: 270, __args=@0x7f940b8684f8: 270) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8684b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 617 (Thread 0x7f92cf9c0640 (LWP 2740278) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b8dd068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b8dd000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b8dd000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b8dd000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=269) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868490: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8684a0: 0x7f956f09f800, __args=@0x7f940b8684a8: 269) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868490: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8684a8: 269, __args=@0x7f940b8684a8: 269) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868468, args=@0x7f940b8684a8: 269, args=@0x7f940b8684a8: 269) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8684a8: 269, __args=@0x7f940b8684a8: 269) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868460) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 616 (Thread 0x7f92d03c1640 (LWP 2740277) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b857d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b857d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b857d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b857d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=268) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868440: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868450: 0x7f956f09f800, __args=@0x7f940b868458: 268) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868440: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868458: 268, __args=@0x7f940b868458: 268) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868418, args=@0x7f940b868458: 268, args=@0x7f940b868458: 268) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868458: 268, __args=@0x7f940b868458: 268) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868410) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 615 (Thread 0x7f92d0dc2640 (LWP 2740276) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b857a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b857a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b857a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b857a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=267) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8683f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868400: 0x7f956f09f800, __args=@0x7f940b868408: 267) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8683f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868408: 267, __args=@0x7f940b868408: 267) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8683c8, args=@0x7f940b868408: 267, args=@0x7f940b868408: 267) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868408: 267, __args=@0x7f940b868408: 267) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8683c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 614 (Thread 0x7f92d17c3640 (LWP 2740275) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b857768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b857700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b857700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b857700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=266) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8683a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8683b0: 0x7f956f09f800, __args=@0x7f940b8683b8: 266) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8683a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8683b8: 266, __args=@0x7f940b8683b8: 266) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868378, args=@0x7f940b8683b8: 266, args=@0x7f940b8683b8: 266) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8683b8: 266, __args=@0x7f940b8683b8: 266) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868370) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 613 (Thread 0x7f92d21c4640 (LWP 2740274) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b857468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b857400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b857400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b857400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=265) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868350: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868360: 0x7f956f09f800, __args=@0x7f940b868368: 265) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868350: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868368: 265, __args=@0x7f940b868368: 265) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868328, args=@0x7f940b868368: 265, args=@0x7f940b868368: 265) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868368: 265, __args=@0x7f940b868368: 265) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 612 (Thread 0x7f92d2bc5640 (LWP 2740273) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b857168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b857100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b857100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b857100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=264) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868300: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868310: 0x7f956f09f800, __args=@0x7f940b868318: 264) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868300: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868318: 264, __args=@0x7f940b868318: 264) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8682d8, args=@0x7f940b868318: 264, args=@0x7f940b868318: 264) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868318: 264, __args=@0x7f940b868318: 264) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8682d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 611 (Thread 0x7f92d35c6640 (LWP 2740272) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b856e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b856e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b856e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b856e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=263) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8682b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8682c0: 0x7f956f09f800, __args=@0x7f940b8682c8: 263) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8682b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8682c8: 263, __args=@0x7f940b8682c8: 263) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868288, args=@0x7f940b8682c8: 263, args=@0x7f940b8682c8: 263) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8682c8: 263, __args=@0x7f940b8682c8: 263) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 610 (Thread 0x7f92d3fc7640 (LWP 2740271) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b856b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b856b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b856b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b856b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=262) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868260: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868270: 0x7f956f09f800, __args=@0x7f940b868278: 262) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868260: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868278: 262, __args=@0x7f940b868278: 262) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868238, args=@0x7f940b868278: 262, args=@0x7f940b868278: 262) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868278: 262, __args=@0x7f940b868278: 262) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868230) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 609 (Thread 0x7f92d49c8640 (LWP 2740270) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b856868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b856800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b856800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b856800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=261) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868210: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868220: 0x7f956f09f800, __args=@0x7f940b868228: 261) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868210: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868228: 261, __args=@0x7f940b868228: 261) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8681e8, args=@0x7f940b868228: 261, args=@0x7f940b868228: 261) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868228: 261, __args=@0x7f940b868228: 261) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8681e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 608 (Thread 0x7f92d53c9640 (LWP 2740269) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b856568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b856500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b856500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b856500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=260) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8681c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8681d0: 0x7f956f09f800, __args=@0x7f940b8681d8: 260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8681c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8681d8: 260, __args=@0x7f940b8681d8: 260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868198, args=@0x7f940b8681d8: 260, args=@0x7f940b8681d8: 260) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8681d8: 260, __args=@0x7f940b8681d8: 260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868190) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 607 (Thread 0x7f92d5dca640 (LWP 2740268) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b856268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b856200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b856200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b856200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=259) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868170: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868180: 0x7f956f09f800, __args=@0x7f940b868188: 259) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868170: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868188: 259, __args=@0x7f940b868188: 259) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868148, args=@0x7f940b868188: 259, args=@0x7f940b868188: 259) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868188: 259, __args=@0x7f940b868188: 259) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 606 (Thread 0x7f92d71cb640 (LWP 2740267) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b855f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=258) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868120: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868130: 0x7f956f09f800, __args=@0x7f940b868138: 258) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868120: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868138: 258, __args=@0x7f940b868138: 258) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8680f8, args=@0x7f940b868138: 258, args=@0x7f940b868138: 258) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868138: 258, __args=@0x7f940b868138: 258) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8680f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 605 (Thread 0x7f92d7bcc640 (LWP 2740266) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b855c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=257) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8680d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b8680e0: 0x7f956f09f800, __args=@0x7f940b8680e8: 257) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b8680d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b8680e8: 257, __args=@0x7f940b8680e8: 257) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b8680a8, args=@0x7f940b8680e8: 257, args=@0x7f940b8680e8: 257) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b8680e8: 257, __args=@0x7f940b8680e8: 257) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b8680a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 604 (Thread 0x7f92d8fcd640 (LWP 2740265) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b855900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=256) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868080: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868090: 0x7f956f09f800, __args=@0x7f940b868098: 256) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868080: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868098: 256, __args=@0x7f940b868098: 256) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868058, args=@0x7f940b868098: 256, args=@0x7f940b868098: 256) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868098: 256, __args=@0x7f940b868098: 256) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868050) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 603 (Thread 0x7f92d99ce640 (LWP 2740264) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b855600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=255) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868030: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b868040: 0x7f956f09f800, __args=@0x7f940b868048: 255) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b868030: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b868048: 255, __args=@0x7f940b868048: 255) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b868008, args=@0x7f940b868048: 255, args=@0x7f940b868048: 255) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b868048: 255, __args=@0x7f940b868048: 255) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b868000) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 602 (Thread 0x7f92da3cf640 (LWP 2740263) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b855300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=254) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024fe0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024ff0: 0x7f956f09f800, __args=@0x7f940b024ff8: 254) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024fe0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024ff8: 254, __args=@0x7f940b024ff8: 254) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024fb8, args=@0x7f940b024ff8: 254, args=@0x7f940b024ff8: 254) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024ff8: 254, __args=@0x7f940b024ff8: 254) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024fb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 601 (Thread 0x7f92dadd0640 (LWP 2740262) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b855068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b855000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b855000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b855000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=253) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024f90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024fa0: 0x7f956f09f800, __args=@0x7f940b024fa8: 253) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024f90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024fa8: 253, __args=@0x7f940b024fa8: 253) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024f68, args=@0x7f940b024fa8: 253, args=@0x7f940b024fa8: 253) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024fa8: 253, __args=@0x7f940b024fa8: 253) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024f60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 600 (Thread 0x7f92db7d1640 (LWP 2740261) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d4d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d4d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d4d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d4d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=252) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024f40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024f50: 0x7f956f09f800, __args=@0x7f940b024f58: 252) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024f40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024f58: 252, __args=@0x7f940b024f58: 252) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024f18, args=@0x7f940b024f58: 252, args=@0x7f940b024f58: 252) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024f58: 252, __args=@0x7f940b024f58: 252) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024f10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 599 (Thread 0x7f92dc1d2640 (LWP 2740260) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d4a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d4a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d4a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d4a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=251) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ef0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024f00: 0x7f956f09f800, __args=@0x7f940b024f08: 251) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ef0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024f08: 251, __args=@0x7f940b024f08: 251) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024ec8, args=@0x7f940b024f08: 251, args=@0x7f940b024f08: 251) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024f08: 251, __args=@0x7f940b024f08: 251) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024ec0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 598 (Thread 0x7f92dcbd3640 (LWP 2740259) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d4768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d4700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d4700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d4700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=250) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ea0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024eb0: 0x7f956f09f800, __args=@0x7f940b024eb8: 250) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ea0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024eb8: 250, __args=@0x7f940b024eb8: 250) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024e78, args=@0x7f940b024eb8: 250, args=@0x7f940b024eb8: 250) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024eb8: 250, __args=@0x7f940b024eb8: 250) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024e70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 597 (Thread 0x7f92dd5d4640 (LWP 2740258) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d4468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d4400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d4400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d4400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=249) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024e50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024e60: 0x7f956f09f800, __args=@0x7f940b024e68: 249) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024e50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024e68: 249, __args=@0x7f940b024e68: 249) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024e28, args=@0x7f940b024e68: 249, args=@0x7f940b024e68: 249) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024e68: 249, __args=@0x7f940b024e68: 249) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024e20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 596 (Thread 0x7f92ddfd5640 (LWP 2740257) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d4168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d4100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d4100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d4100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=248) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024e00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024e10: 0x7f956f09f800, __args=@0x7f940b024e18: 248) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024e00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024e18: 248, __args=@0x7f940b024e18: 248) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024dd8, args=@0x7f940b024e18: 248, args=@0x7f940b024e18: 248) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024e18: 248, __args=@0x7f940b024e18: 248) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024dd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 595 (Thread 0x7f92de9d6640 (LWP 2740256) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d3e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d3e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d3e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d3e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=247) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024db0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024dc0: 0x7f956f09f800, __args=@0x7f940b024dc8: 247) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024db0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024dc8: 247, __args=@0x7f940b024dc8: 247) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024d88, args=@0x7f940b024dc8: 247, args=@0x7f940b024dc8: 247) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024dc8: 247, __args=@0x7f940b024dc8: 247) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024d80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 594 (Thread 0x7f92df3d7640 (LWP 2740255) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d3b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d3b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d3b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d3b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=246) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024d60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024d70: 0x7f956f09f800, __args=@0x7f940b024d78: 246) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024d60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024d78: 246, __args=@0x7f940b024d78: 246) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024d38, args=@0x7f940b024d78: 246, args=@0x7f940b024d78: 246) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024d78: 246, __args=@0x7f940b024d78: 246) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024d30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 593 (Thread 0x7f92dfdd8640 (LWP 2740254) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d3868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d3800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d3800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d3800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=245) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024d10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024d20: 0x7f956f09f800, __args=@0x7f940b024d28: 245) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024d10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024d28: 245, __args=@0x7f940b024d28: 245) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024ce8, args=@0x7f940b024d28: 245, args=@0x7f940b024d28: 245) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024d28: 245, __args=@0x7f940b024d28: 245) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024ce0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 592 (Thread 0x7f92e07d9640 (LWP 2740253) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d3568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d3500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d3500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d3500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=244) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024cc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024cd0: 0x7f956f09f800, __args=@0x7f940b024cd8: 244) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024cc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024cd8: 244, __args=@0x7f940b024cd8: 244) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024c98, args=@0x7f940b024cd8: 244, args=@0x7f940b024cd8: 244) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024cd8: 244, __args=@0x7f940b024cd8: 244) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024c90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 591 (Thread 0x7f92e11da640 (LWP 2740252) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d3268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d3200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d3200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d3200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=243) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024c70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024c80: 0x7f956f09f800, __args=@0x7f940b024c88: 243) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024c70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024c88: 243, __args=@0x7f940b024c88: 243) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024c48, args=@0x7f940b024c88: 243, args=@0x7f940b024c88: 243) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024c88: 243, __args=@0x7f940b024c88: 243) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 590 (Thread 0x7f92e1bdb640 (LWP 2740251) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=242) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024c20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024c30: 0x7f956f09f800, __args=@0x7f940b024c38: 242) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024c20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024c38: 242, __args=@0x7f940b024c38: 242) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024bf8, args=@0x7f940b024c38: 242, args=@0x7f940b024c38: 242) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024c38: 242, __args=@0x7f940b024c38: 242) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024bf0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 589 (Thread 0x7f92e25dc640 (LWP 2740250) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=241) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024bd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024be0: 0x7f956f09f800, __args=@0x7f940b024be8: 241) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024bd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024be8: 241, __args=@0x7f940b024be8: 241) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024ba8, args=@0x7f940b024be8: 241, args=@0x7f940b024be8: 241) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024be8: 241, __args=@0x7f940b024be8: 241) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 588 (Thread 0x7f92e2fdd640 (LWP 2740249) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=240) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024b80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024b90: 0x7f956f09f800, __args=@0x7f940b024b98: 240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024b80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024b98: 240, __args=@0x7f940b024b98: 240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024b58, args=@0x7f940b024b98: 240, args=@0x7f940b024b98: 240) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024b98: 240, __args=@0x7f940b024b98: 240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024b50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 587 (Thread 0x7f92e39de640 (LWP 2740248) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=239) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024b30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024b40: 0x7f956f09f800, __args=@0x7f940b024b48: 239) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024b30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024b48: 239, __args=@0x7f940b024b48: 239) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024b08, args=@0x7f940b024b48: 239, args=@0x7f940b024b48: 239) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024b48: 239, __args=@0x7f940b024b48: 239) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024b00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 586 (Thread 0x7f92e4ddf640 (LWP 2740247) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=238) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ae0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024af0: 0x7f956f09f800, __args=@0x7f940b024af8: 238) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024ae0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024af8: 238, __args=@0x7f940b024af8: 238) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024ab8, args=@0x7f940b024af8: 238, args=@0x7f940b024af8: 238) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024af8: 238, __args=@0x7f940b024af8: 238) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024ab0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 585 (Thread 0x7f92e57e0640 (LWP 2740246) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b7d2068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b7d2000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b7d2000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b7d2000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=237) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024a90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024aa0: 0x7f956f09f800, __args=@0x7f940b024aa8: 237) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024a90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024aa8: 237, __args=@0x7f940b024aa8: 237) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024a68, args=@0x7f940b024aa8: 237, args=@0x7f940b024aa8: 237) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024aa8: 237, __args=@0x7f940b024aa8: 237) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024a60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 584 (Thread 0x7f92e61e1640 (LWP 2740245) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b751d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b751d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b751d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b751d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=236) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024a40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024a50: 0x7f956f09f800, __args=@0x7f940b024a58: 236) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024a40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024a58: 236, __args=@0x7f940b024a58: 236) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024a18, args=@0x7f940b024a58: 236, args=@0x7f940b024a58: 236) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024a58: 236, __args=@0x7f940b024a58: 236) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024a10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 583 (Thread 0x7f92e6be2640 (LWP 2740244) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b751a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b751a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b751a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b751a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=235) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0249f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024a00: 0x7f956f09f800, __args=@0x7f940b024a08: 235) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0249f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024a08: 235, __args=@0x7f940b024a08: 235) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0249c8, args=@0x7f940b024a08: 235, args=@0x7f940b024a08: 235) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024a08: 235, __args=@0x7f940b024a08: 235) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0249c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 582 (Thread 0x7f92e75e3640 (LWP 2740243) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b751768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b751700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b751700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b751700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=234) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0249a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0249b0: 0x7f956f09f800, __args=@0x7f940b0249b8: 234) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0249a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0249b8: 234, __args=@0x7f940b0249b8: 234) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024978, args=@0x7f940b0249b8: 234, args=@0x7f940b0249b8: 234) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0249b8: 234, __args=@0x7f940b0249b8: 234) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024970) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 581 (Thread 0x7f92e7fe4640 (LWP 2740242) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b751468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b751400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b751400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b751400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=233) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024950: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024960: 0x7f956f09f800, __args=@0x7f940b024968: 233) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024950: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024968: 233, __args=@0x7f940b024968: 233) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024928, args=@0x7f940b024968: 233, args=@0x7f940b024968: 233) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024968: 233, __args=@0x7f940b024968: 233) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024920) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 580 (Thread 0x7f92e89e5640 (LWP 2740241) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b751168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b751100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b751100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b751100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=232) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024900: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024910: 0x7f956f09f800, __args=@0x7f940b024918: 232) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024900: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024918: 232, __args=@0x7f940b024918: 232) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0248d8, args=@0x7f940b024918: 232, args=@0x7f940b024918: 232) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024918: 232, __args=@0x7f940b024918: 232) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0248d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 579 (Thread 0x7f92e93e6640 (LWP 2740240) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b750e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b750e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b750e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b750e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=231) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0248b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0248c0: 0x7f956f09f800, __args=@0x7f940b0248c8: 231) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0248b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0248c8: 231, __args=@0x7f940b0248c8: 231) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024888, args=@0x7f940b0248c8: 231, args=@0x7f940b0248c8: 231) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0248c8: 231, __args=@0x7f940b0248c8: 231) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024880) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 578 (Thread 0x7f92e9de7640 (LWP 2740239) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b750b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b750b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b750b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b750b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=230) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024860: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024870: 0x7f956f09f800, __args=@0x7f940b024878: 230) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024860: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024878: 230, __args=@0x7f940b024878: 230) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024838, args=@0x7f940b024878: 230, args=@0x7f940b024878: 230) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024878: 230, __args=@0x7f940b024878: 230) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024830) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 577 (Thread 0x7f92ea7e8640 (LWP 2740238) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b750868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b750800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b750800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b750800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=229) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024810: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024820: 0x7f956f09f800, __args=@0x7f940b024828: 229) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024810: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024828: 229, __args=@0x7f940b024828: 229) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0247e8, args=@0x7f940b024828: 229, args=@0x7f940b024828: 229) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024828: 229, __args=@0x7f940b024828: 229) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0247e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 576 (Thread 0x7f92ebbe9640 (LWP 2740237) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b750568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b750500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b750500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b750500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=228) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0247c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0247d0: 0x7f956f09f800, __args=@0x7f940b0247d8: 228) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0247c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0247d8: 228, __args=@0x7f940b0247d8: 228) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024798, args=@0x7f940b0247d8: 228, args=@0x7f940b0247d8: 228) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0247d8: 228, __args=@0x7f940b0247d8: 228) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024790) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 575 (Thread 0x7f92ec5ea640 (LWP 2740236) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b750268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b750200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b750200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b750200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=227) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024770: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024780: 0x7f956f09f800, __args=@0x7f940b024788: 227) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024770: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024788: 227, __args=@0x7f940b024788: 227) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024748, args=@0x7f940b024788: 227, args=@0x7f940b024788: 227) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024788: 227, __args=@0x7f940b024788: 227) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 574 (Thread 0x7f92ecfeb640 (LWP 2740235) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74ff68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74ff00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b74ff00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74ff00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=226) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024720: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024730: 0x7f956f09f800, __args=@0x7f940b024738: 226) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024720: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024738: 226, __args=@0x7f940b024738: 226) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0246f8, args=@0x7f940b024738: 226, args=@0x7f940b024738: 226) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024738: 226, __args=@0x7f940b024738: 226) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0246f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 573 (Thread 0x7f92ed9ec640 (LWP 2740234) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74fc68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74fc00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b74fc00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74fc00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=225) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0246d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0246e0: 0x7f956f09f800, __args=@0x7f940b0246e8: 225) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0246d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0246e8: 225, __args=@0x7f940b0246e8: 225) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0246a8, args=@0x7f940b0246e8: 225, args=@0x7f940b0246e8: 225) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0246e8: 225, __args=@0x7f940b0246e8: 225) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0246a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 572 (Thread 0x7f92ee3ed640 (LWP 2740233) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74f968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74f900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b74f900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74f900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=224) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024680: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024690: 0x7f956f09f800, __args=@0x7f940b024698: 224) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024680: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024698: 224, __args=@0x7f940b024698: 224) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024658, args=@0x7f940b024698: 224, args=@0x7f940b024698: 224) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024698: 224, __args=@0x7f940b024698: 224) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024650) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 571 (Thread 0x7f92eedee640 (LWP 2740232) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74f668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74f600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b74f600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74f600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=223) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024630: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024640: 0x7f956f09f800, __args=@0x7f940b024648: 223) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024630: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024648: 223, __args=@0x7f940b024648: 223) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024608, args=@0x7f940b024648: 223, args=@0x7f940b024648: 223) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024648: 223, __args=@0x7f940b024648: 223) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 570 (Thread 0x7f92ef7ef640 (LWP 2740231) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74f368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74f300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b74f300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74f300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=222) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0245e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0245f0: 0x7f956f09f800, __args=@0x7f940b0245f8: 222) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0245e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0245f8: 222, __args=@0x7f940b0245f8: 222) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0245b8, args=@0x7f940b0245f8: 222, args=@0x7f940b0245f8: 222) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0245f8: 222, __args=@0x7f940b0245f8: 222) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0245b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 569 (Thread 0x7f92f0bf0640 (LWP 2740230) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b74f068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b74f000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b74f000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b74f000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=221) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024590: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0245a0: 0x7f956f09f800, __args=@0x7f940b0245a8: 221) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024590: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0245a8: 221, __args=@0x7f940b0245a8: 221) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024568, args=@0x7f940b0245a8: 221, args=@0x7f940b0245a8: 221) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0245a8: 221, __args=@0x7f940b0245a8: 221) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024560) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 568 (Thread 0x7f92f1ff1640 (LWP 2740229) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ced68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ced00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ced00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ced00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=220) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024540: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024550: 0x7f956f09f800, __args=@0x7f940b024558: 220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024540: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024558: 220, __args=@0x7f940b024558: 220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024518, args=@0x7f940b024558: 220, args=@0x7f940b024558: 220) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024558: 220, __args=@0x7f940b024558: 220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024510) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 567 (Thread 0x7f92f29f2640 (LWP 2740228) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cea68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cea00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cea00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cea00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=219) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0244f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024500: 0x7f956f09f800, __args=@0x7f940b024508: 219) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0244f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024508: 219, __args=@0x7f940b024508: 219) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0244c8, args=@0x7f940b024508: 219, args=@0x7f940b024508: 219) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024508: 219, __args=@0x7f940b024508: 219) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0244c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 566 (Thread 0x7f92f33f3640 (LWP 2740227) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ce768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ce700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ce700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ce700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=218) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0244a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0244b0: 0x7f956f09f800, __args=@0x7f940b0244b8: 218) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0244a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0244b8: 218, __args=@0x7f940b0244b8: 218) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024478, args=@0x7f940b0244b8: 218, args=@0x7f940b0244b8: 218) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0244b8: 218, __args=@0x7f940b0244b8: 218) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024470) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 565 (Thread 0x7f92f3df4640 (LWP 2740226) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ce468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ce400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ce400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ce400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=217) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024450: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024460: 0x7f956f09f800, __args=@0x7f940b024468: 217) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024450: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024468: 217, __args=@0x7f940b024468: 217) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024428, args=@0x7f940b024468: 217, args=@0x7f940b024468: 217) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024468: 217, __args=@0x7f940b024468: 217) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024420) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 564 (Thread 0x7f92f47f5640 (LWP 2740225) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ce168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ce100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ce100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ce100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=216) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024400: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024410: 0x7f956f09f800, __args=@0x7f940b024418: 216) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024400: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024418: 216, __args=@0x7f940b024418: 216) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0243d8, args=@0x7f940b024418: 216, args=@0x7f940b024418: 216) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024418: 216, __args=@0x7f940b024418: 216) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0243d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 563 (Thread 0x7f92f51f6640 (LWP 2740224) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cde68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cde00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cde00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cde00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=215) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0243b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0243c0: 0x7f956f09f800, __args=@0x7f940b0243c8: 215) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0243b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0243c8: 215, __args=@0x7f940b0243c8: 215) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024388, args=@0x7f940b0243c8: 215, args=@0x7f940b0243c8: 215) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0243c8: 215, __args=@0x7f940b0243c8: 215) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 562 (Thread 0x7f92f5bf7640 (LWP 2740223) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cdb68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cdb00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cdb00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cdb00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=214) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024360: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024370: 0x7f956f09f800, __args=@0x7f940b024378: 214) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024360: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024378: 214, __args=@0x7f940b024378: 214) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024338, args=@0x7f940b024378: 214, args=@0x7f940b024378: 214) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024378: 214, __args=@0x7f940b024378: 214) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024330) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 561 (Thread 0x7f92f65f8640 (LWP 2740222) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cd868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cd800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cd800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cd800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=213) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024310: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024320: 0x7f956f09f800, __args=@0x7f940b024328: 213) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024310: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024328: 213, __args=@0x7f940b024328: 213) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0242e8, args=@0x7f940b024328: 213, args=@0x7f940b024328: 213) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024328: 213, __args=@0x7f940b024328: 213) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0242e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 560 (Thread 0x7f92f6ff9640 (LWP 2740221) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cd568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cd500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cd500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cd500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=212) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0242c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0242d0: 0x7f956f09f800, __args=@0x7f940b0242d8: 212) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0242c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0242d8: 212, __args=@0x7f940b0242d8: 212) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024298, args=@0x7f940b0242d8: 212, args=@0x7f940b0242d8: 212) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0242d8: 212, __args=@0x7f940b0242d8: 212) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024290) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 559 (Thread 0x7f92f79fa640 (LWP 2740220) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cd268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cd200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cd200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cd200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=211) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024270: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024280: 0x7f956f09f800, __args=@0x7f940b024288: 211) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024270: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024288: 211, __args=@0x7f940b024288: 211) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024248, args=@0x7f940b024288: 211, args=@0x7f940b024288: 211) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024288: 211, __args=@0x7f940b024288: 211) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 558 (Thread 0x7f92f83fb640 (LWP 2740219) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ccf68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ccf00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ccf00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ccf00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=210) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024220: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024230: 0x7f956f09f800, __args=@0x7f940b024238: 210) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024220: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024238: 210, __args=@0x7f940b024238: 210) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0241f8, args=@0x7f940b024238: 210, args=@0x7f940b024238: 210) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024238: 210, __args=@0x7f940b024238: 210) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0241f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 557 (Thread 0x7f92f8dfc640 (LWP 2740218) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6ccc68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6ccc00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6ccc00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6ccc00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=209) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0241d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0241e0: 0x7f956f09f800, __args=@0x7f940b0241e8: 209) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0241d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0241e8: 209, __args=@0x7f940b0241e8: 209) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0241a8, args=@0x7f940b0241e8: 209, args=@0x7f940b0241e8: 209) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0241e8: 209, __args=@0x7f940b0241e8: 209) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0241a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 556 (Thread 0x7f92f97fd640 (LWP 2740217) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cc968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cc900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cc900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cc900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=208) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024180: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024190: 0x7f956f09f800, __args=@0x7f940b024198: 208) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024180: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024198: 208, __args=@0x7f940b024198: 208) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024158, args=@0x7f940b024198: 208, args=@0x7f940b024198: 208) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024198: 208, __args=@0x7f940b024198: 208) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024150) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 555 (Thread 0x7f92fa1fe640 (LWP 2740216) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cc668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cc600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cc600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cc600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=207) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024130: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024140: 0x7f956f09f800, __args=@0x7f940b024148: 207) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024130: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024148: 207, __args=@0x7f940b024148: 207) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024108, args=@0x7f940b024148: 207, args=@0x7f940b024148: 207) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024148: 207, __args=@0x7f940b024148: 207) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 554 (Thread 0x7f92fabff640 (LWP 2740215) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cc368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cc300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cc300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cc300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=206) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0240e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0240f0: 0x7f956f09f800, __args=@0x7f940b0240f8: 206) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0240e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0240f8: 206, __args=@0x7f940b0240f8: 206) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0240b8, args=@0x7f940b0240f8: 206, args=@0x7f940b0240f8: 206) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0240f8: 206, __args=@0x7f940b0240f8: 206) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0240b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 553 (Thread 0x7f92fcca0640 (LWP 2740214) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b6cc068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b6cc000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b6cc000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b6cc000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=205) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024090: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0240a0: 0x7f956f09f800, __args=@0x7f940b0240a8: 205) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024090: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0240a8: 205, __args=@0x7f940b0240a8: 205) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024068, args=@0x7f940b0240a8: 205, args=@0x7f940b0240a8: 205) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0240a8: 205, __args=@0x7f940b0240a8: 205) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024060) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 552 (Thread 0x7f92fe0a1640 (LWP 2740213) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64bd68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64bd00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b64bd00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64bd00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=204) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024040: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024050: 0x7f956f09f800, __args=@0x7f940b024058: 204) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b024040: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024058: 204, __args=@0x7f940b024058: 204) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b024018, args=@0x7f940b024058: 204, args=@0x7f940b024058: 204) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024058: 204, __args=@0x7f940b024058: 204) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b024010) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 551 (Thread 0x7f92feaa2640 (LWP 2740212) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64ba68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64ba00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b64ba00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64ba00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=203) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023ff0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b024000: 0x7f956f09f800, __args=@0x7f940b024008: 203) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023ff0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b024008: 203, __args=@0x7f940b024008: 203) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023fc8, args=@0x7f940b024008: 203, args=@0x7f940b024008: 203) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b024008: 203, __args=@0x7f940b024008: 203) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023fc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 550 (Thread 0x7f92ff4a3640 (LWP 2740211) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64b768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64b700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b64b700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64b700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=202) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023fa0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023fb0: 0x7f956f09f800, __args=@0x7f940b023fb8: 202) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023fa0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023fb8: 202, __args=@0x7f940b023fb8: 202) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023f78, args=@0x7f940b023fb8: 202, args=@0x7f940b023fb8: 202) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023fb8: 202, __args=@0x7f940b023fb8: 202) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023f70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 549 (Thread 0x7f92ffea4640 (LWP 2740210) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64b468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64b400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b64b400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64b400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=201) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023f50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023f60: 0x7f956f09f800, __args=@0x7f940b023f68: 201) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023f50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023f68: 201, __args=@0x7f940b023f68: 201) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023f28, args=@0x7f940b023f68: 201, args=@0x7f940b023f68: 201) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023f68: 201, __args=@0x7f940b023f68: 201) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023f20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 548 (Thread 0x7f93008a5640 (LWP 2740209) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64b168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64b100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b64b100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64b100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=200) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023f00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023f10: 0x7f956f09f800, __args=@0x7f940b023f18: 200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023f00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023f18: 200, __args=@0x7f940b023f18: 200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023ed8, args=@0x7f940b023f18: 200, args=@0x7f940b023f18: 200) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023f18: 200, __args=@0x7f940b023f18: 200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023ed0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 547 (Thread 0x7f93012a6640 (LWP 2740208) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64ae68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64ae00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b64ae00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64ae00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=199) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023eb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023ec0: 0x7f956f09f800, __args=@0x7f940b023ec8: 199) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023eb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023ec8: 199, __args=@0x7f940b023ec8: 199) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023e88, args=@0x7f940b023ec8: 199, args=@0x7f940b023ec8: 199) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023ec8: 199, __args=@0x7f940b023ec8: 199) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023e80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 546 (Thread 0x7f9301ca7640 (LWP 2740207) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64ab68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64ab00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b64ab00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64ab00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=198) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023e60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023e70: 0x7f956f09f800, __args=@0x7f940b023e78: 198) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023e60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023e78: 198, __args=@0x7f940b023e78: 198) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023e38, args=@0x7f940b023e78: 198, args=@0x7f940b023e78: 198) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023e78: 198, __args=@0x7f940b023e78: 198) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023e30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 545 (Thread 0x7f93026a8640 (LWP 2740206) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64a868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64a800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b64a800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64a800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=197) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023e10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023e20: 0x7f956f09f800, __args=@0x7f940b023e28: 197) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023e10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023e28: 197, __args=@0x7f940b023e28: 197) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023de8, args=@0x7f940b023e28: 197, args=@0x7f940b023e28: 197) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023e28: 197, __args=@0x7f940b023e28: 197) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023de0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 544 (Thread 0x7f93030a9640 (LWP 2740205) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64a568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64a500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b64a500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64a500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=196) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023dc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023dd0: 0x7f956f09f800, __args=@0x7f940b023dd8: 196) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023dc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023dd8: 196, __args=@0x7f940b023dd8: 196) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023d98, args=@0x7f940b023dd8: 196, args=@0x7f940b023dd8: 196) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023dd8: 196, __args=@0x7f940b023dd8: 196) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023d90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 543 (Thread 0x7f9303aaa640 (LWP 2740204) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b64a268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b64a200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b64a200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b64a200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=195) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023d70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023d80: 0x7f956f09f800, __args=@0x7f940b023d88: 195) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023d70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023d88: 195, __args=@0x7f940b023d88: 195) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023d48, args=@0x7f940b023d88: 195, args=@0x7f940b023d88: 195) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023d88: 195, __args=@0x7f940b023d88: 195) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023d40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 542 (Thread 0x7f93044ab640 (LWP 2740203) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b649f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=194) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023d20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023d30: 0x7f956f09f800, __args=@0x7f940b023d38: 194) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023d20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023d38: 194, __args=@0x7f940b023d38: 194) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023cf8, args=@0x7f940b023d38: 194, args=@0x7f940b023d38: 194) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023d38: 194, __args=@0x7f940b023d38: 194) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023cf0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 541 (Thread 0x7f9304eac640 (LWP 2740202) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b649c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=193) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023cd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023ce0: 0x7f956f09f800, __args=@0x7f940b023ce8: 193) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023cd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023ce8: 193, __args=@0x7f940b023ce8: 193) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023ca8, args=@0x7f940b023ce8: 193, args=@0x7f940b023ce8: 193) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023ce8: 193, __args=@0x7f940b023ce8: 193) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023ca0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 540 (Thread 0x7f93058ad640 (LWP 2740201) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b649900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=192) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023c80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023c90: 0x7f956f09f800, __args=@0x7f940b023c98: 192) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023c80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023c98: 192, __args=@0x7f940b023c98: 192) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023c58, args=@0x7f940b023c98: 192, args=@0x7f940b023c98: 192) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023c98: 192, __args=@0x7f940b023c98: 192) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023c50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 539 (Thread 0x7f93062ae640 (LWP 2740200) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b649600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=191) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023c30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023c40: 0x7f956f09f800, __args=@0x7f940b023c48: 191) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023c30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023c48: 191, __args=@0x7f940b023c48: 191) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023c08, args=@0x7f940b023c48: 191, args=@0x7f940b023c48: 191) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023c48: 191, __args=@0x7f940b023c48: 191) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023c00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 538 (Thread 0x7f9306caf640 (LWP 2740199) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b649300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=190) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023be0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023bf0: 0x7f956f09f800, __args=@0x7f940b023bf8: 190) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023be0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023bf8: 190, __args=@0x7f940b023bf8: 190) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023bb8, args=@0x7f940b023bf8: 190, args=@0x7f940b023bf8: 190) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023bf8: 190, __args=@0x7f940b023bf8: 190) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023bb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 537 (Thread 0x7f93076b0640 (LWP 2740198) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b649068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b649000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b649000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b649000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=189) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023b90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023ba0: 0x7f956f09f800, __args=@0x7f940b023ba8: 189) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023b90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023ba8: 189, __args=@0x7f940b023ba8: 189) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023b68, args=@0x7f940b023ba8: 189, args=@0x7f940b023ba8: 189) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023ba8: 189, __args=@0x7f940b023ba8: 189) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023b60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 536 (Thread 0x7f93080b1640 (LWP 2740197) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c8d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c8d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c8d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c8d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=188) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023b40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023b50: 0x7f956f09f800, __args=@0x7f940b023b58: 188) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023b40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023b58: 188, __args=@0x7f940b023b58: 188) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023b18, args=@0x7f940b023b58: 188, args=@0x7f940b023b58: 188) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023b58: 188, __args=@0x7f940b023b58: 188) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023b10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 535 (Thread 0x7f9308ab2640 (LWP 2740196) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c8a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c8a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c8a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c8a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=187) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023af0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023b00: 0x7f956f09f800, __args=@0x7f940b023b08: 187) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023af0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023b08: 187, __args=@0x7f940b023b08: 187) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023ac8, args=@0x7f940b023b08: 187, args=@0x7f940b023b08: 187) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023b08: 187, __args=@0x7f940b023b08: 187) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023ac0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 534 (Thread 0x7f93094b3640 (LWP 2740195) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c8768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c8700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c8700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c8700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=186) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023aa0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023ab0: 0x7f956f09f800, __args=@0x7f940b023ab8: 186) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023aa0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023ab8: 186, __args=@0x7f940b023ab8: 186) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023a78, args=@0x7f940b023ab8: 186, args=@0x7f940b023ab8: 186) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023ab8: 186, __args=@0x7f940b023ab8: 186) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023a70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 533 (Thread 0x7f9309eb4640 (LWP 2740194) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c8468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c8400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c8400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c8400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=185) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023a50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023a60: 0x7f956f09f800, __args=@0x7f940b023a68: 185) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023a50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023a68: 185, __args=@0x7f940b023a68: 185) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023a28, args=@0x7f940b023a68: 185, args=@0x7f940b023a68: 185) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023a68: 185, __args=@0x7f940b023a68: 185) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023a20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 532 (Thread 0x7f930a8b5640 (LWP 2740193) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c8168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c8100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c8100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c8100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=184) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023a00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023a10: 0x7f956f09f800, __args=@0x7f940b023a18: 184) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023a00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023a18: 184, __args=@0x7f940b023a18: 184) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0239d8, args=@0x7f940b023a18: 184, args=@0x7f940b023a18: 184) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023a18: 184, __args=@0x7f940b023a18: 184) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0239d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 531 (Thread 0x7f930b2b6640 (LWP 2740192) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c7e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c7e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c7e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c7e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=183) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0239b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0239c0: 0x7f956f09f800, __args=@0x7f940b0239c8: 183) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0239b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0239c8: 183, __args=@0x7f940b0239c8: 183) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023988, args=@0x7f940b0239c8: 183, args=@0x7f940b0239c8: 183) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0239c8: 183, __args=@0x7f940b0239c8: 183) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023980) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 530 (Thread 0x7f930c6b7640 (LWP 2740191) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c7b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c7b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c7b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c7b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=182) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023960: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023970: 0x7f956f09f800, __args=@0x7f940b023978: 182) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023960: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023978: 182, __args=@0x7f940b023978: 182) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023938, args=@0x7f940b023978: 182, args=@0x7f940b023978: 182) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023978: 182, __args=@0x7f940b023978: 182) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023930) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 529 (Thread 0x7f930d0b8640 (LWP 2740190) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c7868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c7800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c7800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c7800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=181) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023910: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023920: 0x7f956f09f800, __args=@0x7f940b023928: 181) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023910: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023928: 181, __args=@0x7f940b023928: 181) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0238e8, args=@0x7f940b023928: 181, args=@0x7f940b023928: 181) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023928: 181, __args=@0x7f940b023928: 181) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0238e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 528 (Thread 0x7f930dab9640 (LWP 2740189) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c7568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c7500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c7500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c7500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=180) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0238c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0238d0: 0x7f956f09f800, __args=@0x7f940b0238d8: 180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0238c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0238d8: 180, __args=@0x7f940b0238d8: 180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023898, args=@0x7f940b0238d8: 180, args=@0x7f940b0238d8: 180) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0238d8: 180, __args=@0x7f940b0238d8: 180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023890) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 527 (Thread 0x7f930e4ba640 (LWP 2740188) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c7268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c7200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c7200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c7200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=179) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023870: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023880: 0x7f956f09f800, __args=@0x7f940b023888: 179) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023870: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023888: 179, __args=@0x7f940b023888: 179) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023848, args=@0x7f940b023888: 179, args=@0x7f940b023888: 179) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023888: 179, __args=@0x7f940b023888: 179) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 526 (Thread 0x7f930eebb640 (LWP 2740187) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=178) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023820: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023830: 0x7f956f09f800, __args=@0x7f940b023838: 178) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023820: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023838: 178, __args=@0x7f940b023838: 178) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0237f8, args=@0x7f940b023838: 178, args=@0x7f940b023838: 178) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023838: 178, __args=@0x7f940b023838: 178) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0237f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 525 (Thread 0x7f930fdbc640 (LWP 2740186) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=177) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0237d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0237e0: 0x7f956f09f800, __args=@0x7f940b0237e8: 177) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0237d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0237e8: 177, __args=@0x7f940b0237e8: 177) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0237a8, args=@0x7f940b0237e8: 177, args=@0x7f940b0237e8: 177) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0237e8: 177, __args=@0x7f940b0237e8: 177) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0237a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 524 (Thread 0x7f93107bd640 (LWP 2740185) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=176) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023780: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023790: 0x7f956f09f800, __args=@0x7f940b023798: 176) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023780: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023798: 176, __args=@0x7f940b023798: 176) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023758, args=@0x7f940b023798: 176, args=@0x7f940b023798: 176) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023798: 176, __args=@0x7f940b023798: 176) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023750) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 523 (Thread 0x7f93111be640 (LWP 2740184) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=175) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023730: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023740: 0x7f956f09f800, __args=@0x7f940b023748: 175) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023730: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023748: 175, __args=@0x7f940b023748: 175) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023708, args=@0x7f940b023748: 175, args=@0x7f940b023748: 175) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023748: 175, __args=@0x7f940b023748: 175) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023700) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 522 (Thread 0x7f9311bbf640 (LWP 2740183) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=174) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0236e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0236f0: 0x7f956f09f800, __args=@0x7f940b0236f8: 174) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0236e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0236f8: 174, __args=@0x7f940b0236f8: 174) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0236b8, args=@0x7f940b0236f8: 174, args=@0x7f940b0236f8: 174) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0236f8: 174, __args=@0x7f940b0236f8: 174) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0236b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 521 (Thread 0x7f93125c0640 (LWP 2740182) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b5c6068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b5c6000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b5c6000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b5c6000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=173) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023690: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0236a0: 0x7f956f09f800, __args=@0x7f940b0236a8: 173) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023690: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0236a8: 173, __args=@0x7f940b0236a8: 173) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023668, args=@0x7f940b0236a8: 173, args=@0x7f940b0236a8: 173) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0236a8: 173, __args=@0x7f940b0236a8: 173) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023660) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 520 (Thread 0x7f9312fc1640 (LWP 2740181) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b545d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b545d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b545d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b545d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=172) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023640: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023650: 0x7f956f09f800, __args=@0x7f940b023658: 172) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023640: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023658: 172, __args=@0x7f940b023658: 172) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023618, args=@0x7f940b023658: 172, args=@0x7f940b023658: 172) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023658: 172, __args=@0x7f940b023658: 172) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023610) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 519 (Thread 0x7f93139c2640 (LWP 2740180) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b545a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b545a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b545a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b545a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=171) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0235f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023600: 0x7f956f09f800, __args=@0x7f940b023608: 171) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0235f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023608: 171, __args=@0x7f940b023608: 171) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0235c8, args=@0x7f940b023608: 171, args=@0x7f940b023608: 171) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023608: 171, __args=@0x7f940b023608: 171) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0235c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 518 (Thread 0x7f93143c3640 (LWP 2740179) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b545768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b545700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b545700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b545700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=170) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0235a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0235b0: 0x7f956f09f800, __args=@0x7f940b0235b8: 170) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0235a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0235b8: 170, __args=@0x7f940b0235b8: 170) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023578, args=@0x7f940b0235b8: 170, args=@0x7f940b0235b8: 170) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0235b8: 170, __args=@0x7f940b0235b8: 170) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023570) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 517 (Thread 0x7f9314dc4640 (LWP 2740177) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b545468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b545400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b545400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b545400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=169) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023550: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023560: 0x7f956f09f800, __args=@0x7f940b023568: 169) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023550: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023568: 169, __args=@0x7f940b023568: 169) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023528, args=@0x7f940b023568: 169, args=@0x7f940b023568: 169) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023568: 169, __args=@0x7f940b023568: 169) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023520) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 516 (Thread 0x7f93157c5640 (LWP 2740176) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b545168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b545100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b545100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b545100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023500: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023510: 0x7f956f09f800, __args=@0x7f940b023518: 168) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023500: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023518: 168, __args=@0x7f940b023518: 168) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0234d8, args=@0x7f940b023518: 168, args=@0x7f940b023518: 168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023518: 168, __args=@0x7f940b023518: 168) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0234d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 515 (Thread 0x7f93161c6640 (LWP 2740174) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b544e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b544e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b544e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b544e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=167) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0234b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0234c0: 0x7f956f09f800, __args=@0x7f940b0234c8: 167) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0234b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0234c8: 167, __args=@0x7f940b0234c8: 167) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023488, args=@0x7f940b0234c8: 167, args=@0x7f940b0234c8: 167) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0234c8: 167, __args=@0x7f940b0234c8: 167) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 514 (Thread 0x7f93171c7640 (LWP 2740172) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b544b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b544b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b544b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b544b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=166) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023460: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023470: 0x7f956f09f800, __args=@0x7f940b023478: 166) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023460: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023478: 166, __args=@0x7f940b023478: 166) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023438, args=@0x7f940b023478: 166, args=@0x7f940b023478: 166) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023478: 166, __args=@0x7f940b023478: 166) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023430) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 513 (Thread 0x7f9317bc8640 (LWP 2740171) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b544868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b544800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b544800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b544800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=165) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023410: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023420: 0x7f956f09f800, __args=@0x7f940b023428: 165) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023410: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023428: 165, __args=@0x7f940b023428: 165) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0233e8, args=@0x7f940b023428: 165, args=@0x7f940b023428: 165) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023428: 165, __args=@0x7f940b023428: 165) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0233e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 512 (Thread 0x7f93185c9640 (LWP 2740170) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b544568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b544500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b544500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b544500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=164) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0233c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0233d0: 0x7f956f09f800, __args=@0x7f940b0233d8: 164) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0233c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0233d8: 164, __args=@0x7f940b0233d8: 164) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023398, args=@0x7f940b0233d8: 164, args=@0x7f940b0233d8: 164) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0233d8: 164, __args=@0x7f940b0233d8: 164) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023390) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 511 (Thread 0x7f9318fca640 (LWP 2740168) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b544268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b544200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b544200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b544200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=163) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023370: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023380: 0x7f956f09f800, __args=@0x7f940b023388: 163) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023370: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023388: 163, __args=@0x7f940b023388: 163) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023348, args=@0x7f940b023388: 163, args=@0x7f940b023388: 163) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023388: 163, __args=@0x7f940b023388: 163) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 510 (Thread 0x7f93199cb640 (LWP 2740166) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b543f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=162) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023320: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023330: 0x7f956f09f800, __args=@0x7f940b023338: 162) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023320: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023338: 162, __args=@0x7f940b023338: 162) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0232f8, args=@0x7f940b023338: 162, args=@0x7f940b023338: 162) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023338: 162, __args=@0x7f940b023338: 162) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0232f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 509 (Thread 0x7f931a3cc640 (LWP 2740164) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b543c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=161) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0232d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0232e0: 0x7f956f09f800, __args=@0x7f940b0232e8: 161) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0232d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0232e8: 161, __args=@0x7f940b0232e8: 161) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0232a8, args=@0x7f940b0232e8: 161, args=@0x7f940b0232e8: 161) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0232e8: 161, __args=@0x7f940b0232e8: 161) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0232a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 508 (Thread 0x7f931adcd640 (LWP 2740163) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b543900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=160) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023280: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023290: 0x7f956f09f800, __args=@0x7f940b023298: 160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023280: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023298: 160, __args=@0x7f940b023298: 160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023258, args=@0x7f940b023298: 160, args=@0x7f940b023298: 160) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023298: 160, __args=@0x7f940b023298: 160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023250) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 507 (Thread 0x7f931b7ce640 (LWP 2740162) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b543600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=159) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023230: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023240: 0x7f956f09f800, __args=@0x7f940b023248: 159) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023230: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023248: 159, __args=@0x7f940b023248: 159) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023208, args=@0x7f940b023248: 159, args=@0x7f940b023248: 159) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023248: 159, __args=@0x7f940b023248: 159) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 506 (Thread 0x7f931c1cf640 (LWP 2740161) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b543300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=158) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0231e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0231f0: 0x7f956f09f800, __args=@0x7f940b0231f8: 158) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0231e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0231f8: 158, __args=@0x7f940b0231f8: 158) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0231b8, args=@0x7f940b0231f8: 158, args=@0x7f940b0231f8: 158) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0231f8: 158, __args=@0x7f940b0231f8: 158) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0231b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 505 (Thread 0x7f931cbd0640 (LWP 2740160) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b543068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b543000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b543000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b543000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=157) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023190: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0231a0: 0x7f956f09f800, __args=@0x7f940b0231a8: 157) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023190: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0231a8: 157, __args=@0x7f940b0231a8: 157) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023168, args=@0x7f940b0231a8: 157, args=@0x7f940b0231a8: 157) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0231a8: 157, __args=@0x7f940b0231a8: 157) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 504 (Thread 0x7f931d5d1640 (LWP 2740159) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c2d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c2d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c2d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c2d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=156) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023140: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023150: 0x7f956f09f800, __args=@0x7f940b023158: 156) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023140: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023158: 156, __args=@0x7f940b023158: 156) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023118, args=@0x7f940b023158: 156, args=@0x7f940b023158: 156) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023158: 156, __args=@0x7f940b023158: 156) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023110) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 503 (Thread 0x7f931dfd2640 (LWP 2740157) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c2a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c2a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c2a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c2a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=155) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0230f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023100: 0x7f956f09f800, __args=@0x7f940b023108: 155) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0230f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023108: 155, __args=@0x7f940b023108: 155) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0230c8, args=@0x7f940b023108: 155, args=@0x7f940b023108: 155) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023108: 155, __args=@0x7f940b023108: 155) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0230c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 502 (Thread 0x7f931e9d3640 (LWP 2740156) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c2768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c2700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c2700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c2700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=154) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0230a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0230b0: 0x7f956f09f800, __args=@0x7f940b0230b8: 154) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0230a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0230b8: 154, __args=@0x7f940b0230b8: 154) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023078, args=@0x7f940b0230b8: 154, args=@0x7f940b0230b8: 154) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0230b8: 154, __args=@0x7f940b0230b8: 154) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023070) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 501 (Thread 0x7f931f3d4640 (LWP 2740155) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c2468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c2400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c2400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c2400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=153) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023050: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023060: 0x7f956f09f800, __args=@0x7f940b023068: 153) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023050: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023068: 153, __args=@0x7f940b023068: 153) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b023028, args=@0x7f940b023068: 153, args=@0x7f940b023068: 153) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023068: 153, __args=@0x7f940b023068: 153) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b023020) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 500 (Thread 0x7f931fdd5640 (LWP 2740154) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c2168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c2100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c2100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c2100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=152) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023000: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b023010: 0x7f956f09f800, __args=@0x7f940b023018: 152) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b023000: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b023018: 152, __args=@0x7f940b023018: 152) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022fd8, args=@0x7f940b023018: 152, args=@0x7f940b023018: 152) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b023018: 152, __args=@0x7f940b023018: 152) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022fd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 499 (Thread 0x7f93207d6640 (LWP 2740153) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c1e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c1e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c1e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c1e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=151) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022fb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022fc0: 0x7f956f09f800, __args=@0x7f940b022fc8: 151) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022fb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022fc8: 151, __args=@0x7f940b022fc8: 151) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022f88, args=@0x7f940b022fc8: 151, args=@0x7f940b022fc8: 151) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022fc8: 151, __args=@0x7f940b022fc8: 151) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022f80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 498 (Thread 0x7f93211d7640 (LWP 2740152) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c1b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c1b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c1b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c1b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=150) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022f60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022f70: 0x7f956f09f800, __args=@0x7f940b022f78: 150) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022f60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022f78: 150, __args=@0x7f940b022f78: 150) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022f38, args=@0x7f940b022f78: 150, args=@0x7f940b022f78: 150) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022f78: 150, __args=@0x7f940b022f78: 150) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022f30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 497 (Thread 0x7f9321bd8640 (LWP 2740151) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c1868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c1800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c1800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c1800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=149) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022f10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022f20: 0x7f956f09f800, __args=@0x7f940b022f28: 149) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022f10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022f28: 149, __args=@0x7f940b022f28: 149) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022ee8, args=@0x7f940b022f28: 149, args=@0x7f940b022f28: 149) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022f28: 149, __args=@0x7f940b022f28: 149) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022ee0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 496 (Thread 0x7f93225d9640 (LWP 2740150) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c1568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c1500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c1500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c1500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=148) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ec0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022ed0: 0x7f956f09f800, __args=@0x7f940b022ed8: 148) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ec0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022ed8: 148, __args=@0x7f940b022ed8: 148) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022e98, args=@0x7f940b022ed8: 148, args=@0x7f940b022ed8: 148) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022ed8: 148, __args=@0x7f940b022ed8: 148) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022e90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 495 (Thread 0x7f9322fda640 (LWP 2740149) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c1268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c1200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c1200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c1200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=147) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022e70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022e80: 0x7f956f09f800, __args=@0x7f940b022e88: 147) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022e70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022e88: 147, __args=@0x7f940b022e88: 147) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022e48, args=@0x7f940b022e88: 147, args=@0x7f940b022e88: 147) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022e88: 147, __args=@0x7f940b022e88: 147) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022e40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 494 (Thread 0x7f93239db640 (LWP 2740148) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=146) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022e20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022e30: 0x7f956f09f800, __args=@0x7f940b022e38: 146) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022e20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022e38: 146, __args=@0x7f940b022e38: 146) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022df8, args=@0x7f940b022e38: 146, args=@0x7f940b022e38: 146) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022e38: 146, __args=@0x7f940b022e38: 146) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022df0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 493 (Thread 0x7f93243dc640 (LWP 2740147) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=145) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022dd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022de0: 0x7f956f09f800, __args=@0x7f940b022de8: 145) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022dd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022de8: 145, __args=@0x7f940b022de8: 145) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022da8, args=@0x7f940b022de8: 145, args=@0x7f940b022de8: 145) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022de8: 145, __args=@0x7f940b022de8: 145) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022da0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 492 (Thread 0x7f9324ddd640 (LWP 2740146) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=144) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022d80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022d90: 0x7f956f09f800, __args=@0x7f940b022d98: 144) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022d80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022d98: 144, __args=@0x7f940b022d98: 144) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022d58, args=@0x7f940b022d98: 144, args=@0x7f940b022d98: 144) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022d98: 144, __args=@0x7f940b022d98: 144) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022d50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 491 (Thread 0x7f93257de640 (LWP 2740145) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=143) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022d30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022d40: 0x7f956f09f800, __args=@0x7f940b022d48: 143) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022d30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022d48: 143, __args=@0x7f940b022d48: 143) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022d08, args=@0x7f940b022d48: 143, args=@0x7f940b022d48: 143) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022d48: 143, __args=@0x7f940b022d48: 143) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022d00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 490 (Thread 0x7f93261df640 (LWP 2740144) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=142) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ce0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022cf0: 0x7f956f09f800, __args=@0x7f940b022cf8: 142) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ce0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022cf8: 142, __args=@0x7f940b022cf8: 142) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022cb8, args=@0x7f940b022cf8: 142, args=@0x7f940b022cf8: 142) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022cf8: 142, __args=@0x7f940b022cf8: 142) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022cb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 489 (Thread 0x7f9326be0640 (LWP 2740143) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b4c0068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b4c0000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b4c0000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b4c0000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=141) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022c90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022ca0: 0x7f956f09f800, __args=@0x7f940b022ca8: 141) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022c90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022ca8: 141, __args=@0x7f940b022ca8: 141) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022c68, args=@0x7f940b022ca8: 141, args=@0x7f940b022ca8: 141) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022ca8: 141, __args=@0x7f940b022ca8: 141) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022c60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 488 (Thread 0x7f93275e1640 (LWP 2740142) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43fd68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43fd00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43fd00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43fd00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=140) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022c40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022c50: 0x7f956f09f800, __args=@0x7f940b022c58: 140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022c40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022c58: 140, __args=@0x7f940b022c58: 140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022c18, args=@0x7f940b022c58: 140, args=@0x7f940b022c58: 140) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022c58: 140, __args=@0x7f940b022c58: 140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022c10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 487 (Thread 0x7f93287e2640 (LWP 2740141) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43fa68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43fa00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43fa00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43fa00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=139) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022bf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022c00: 0x7f956f09f800, __args=@0x7f940b022c08: 139) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022bf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022c08: 139, __args=@0x7f940b022c08: 139) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022bc8, args=@0x7f940b022c08: 139, args=@0x7f940b022c08: 139) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022c08: 139, __args=@0x7f940b022c08: 139) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022bc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 486 (Thread 0x7f93291e3640 (LWP 2740140) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43f768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43f700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43f700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43f700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=138) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ba0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022bb0: 0x7f956f09f800, __args=@0x7f940b022bb8: 138) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ba0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022bb8: 138, __args=@0x7f940b022bb8: 138) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022b78, args=@0x7f940b022bb8: 138, args=@0x7f940b022bb8: 138) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022bb8: 138, __args=@0x7f940b022bb8: 138) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022b70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 485 (Thread 0x7f9329be4640 (LWP 2740139) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43f468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43f400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43f400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43f400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=137) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022b50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022b60: 0x7f956f09f800, __args=@0x7f940b022b68: 137) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022b50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022b68: 137, __args=@0x7f940b022b68: 137) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022b28, args=@0x7f940b022b68: 137, args=@0x7f940b022b68: 137) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022b68: 137, __args=@0x7f940b022b68: 137) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022b20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 484 (Thread 0x7f932a5e5640 (LWP 2740138) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43f168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43f100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43f100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43f100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=136) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022b00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022b10: 0x7f956f09f800, __args=@0x7f940b022b18: 136) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022b00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022b18: 136, __args=@0x7f940b022b18: 136) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022ad8, args=@0x7f940b022b18: 136, args=@0x7f940b022b18: 136) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022b18: 136, __args=@0x7f940b022b18: 136) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022ad0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 483 (Thread 0x7f932afe6640 (LWP 2740137) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43ee68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43ee00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43ee00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43ee00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=135) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ab0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022ac0: 0x7f956f09f800, __args=@0x7f940b022ac8: 135) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022ab0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022ac8: 135, __args=@0x7f940b022ac8: 135) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022a88, args=@0x7f940b022ac8: 135, args=@0x7f940b022ac8: 135) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022ac8: 135, __args=@0x7f940b022ac8: 135) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 482 (Thread 0x7f932b9e7640 (LWP 2740136) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43eb68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43eb00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43eb00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43eb00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=134) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022a60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022a70: 0x7f956f09f800, __args=@0x7f940b022a78: 134) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022a60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022a78: 134, __args=@0x7f940b022a78: 134) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022a38, args=@0x7f940b022a78: 134, args=@0x7f940b022a78: 134) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022a78: 134, __args=@0x7f940b022a78: 134) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022a30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 481 (Thread 0x7f932c3e8640 (LWP 2740135) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43e868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43e800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43e800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43e800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=133) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022a10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022a20: 0x7f956f09f800, __args=@0x7f940b022a28: 133) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022a10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022a28: 133, __args=@0x7f940b022a28: 133) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0229e8, args=@0x7f940b022a28: 133, args=@0x7f940b022a28: 133) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022a28: 133, __args=@0x7f940b022a28: 133) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0229e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 480 (Thread 0x7f932cde9640 (LWP 2740134) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43e568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43e500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43e500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43e500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=132) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0229c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0229d0: 0x7f956f09f800, __args=@0x7f940b0229d8: 132) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0229c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0229d8: 132, __args=@0x7f940b0229d8: 132) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022998, args=@0x7f940b0229d8: 132, args=@0x7f940b0229d8: 132) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0229d8: 132, __args=@0x7f940b0229d8: 132) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022990) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 479 (Thread 0x7f932d7ea640 (LWP 2740133) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43e268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43e200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43e200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43e200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=131) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022970: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022980: 0x7f956f09f800, __args=@0x7f940b022988: 131) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022970: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022988: 131, __args=@0x7f940b022988: 131) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022948, args=@0x7f940b022988: 131, args=@0x7f940b022988: 131) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022988: 131, __args=@0x7f940b022988: 131) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 478 (Thread 0x7f932e1eb640 (LWP 2740132) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43df68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43df00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43df00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43df00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=130) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022920: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022930: 0x7f956f09f800, __args=@0x7f940b022938: 130) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022920: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022938: 130, __args=@0x7f940b022938: 130) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0228f8, args=@0x7f940b022938: 130, args=@0x7f940b022938: 130) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022938: 130, __args=@0x7f940b022938: 130) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0228f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 477 (Thread 0x7f932ebec640 (LWP 2740131) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43dc68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43dc00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43dc00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43dc00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=129) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0228d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0228e0: 0x7f956f09f800, __args=@0x7f940b0228e8: 129) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0228d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0228e8: 129, __args=@0x7f940b0228e8: 129) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0228a8, args=@0x7f940b0228e8: 129, args=@0x7f940b0228e8: 129) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0228e8: 129, __args=@0x7f940b0228e8: 129) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0228a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 476 (Thread 0x7f932f5ed640 (LWP 2740130) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43d968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43d900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43d900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43d900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=128) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022880: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022890: 0x7f956f09f800, __args=@0x7f940b022898: 128) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022880: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022898: 128, __args=@0x7f940b022898: 128) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022858, args=@0x7f940b022898: 128, args=@0x7f940b022898: 128) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022898: 128, __args=@0x7f940b022898: 128) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022850) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 475 (Thread 0x7f932ffee640 (LWP 2740129) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43d668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43d600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43d600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43d600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=127) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022830: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022840: 0x7f956f09f800, __args=@0x7f940b022848: 127) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022830: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022848: 127, __args=@0x7f940b022848: 127) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022808, args=@0x7f940b022848: 127, args=@0x7f940b022848: 127) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022848: 127, __args=@0x7f940b022848: 127) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022800) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 474 (Thread 0x7f93309ef640 (LWP 2740128) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43d368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43d300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43d300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43d300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=126) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0227e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0227f0: 0x7f956f09f800, __args=@0x7f940b0227f8: 126) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0227e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0227f8: 126, __args=@0x7f940b0227f8: 126) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0227b8, args=@0x7f940b0227f8: 126, args=@0x7f940b0227f8: 126) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0227f8: 126, __args=@0x7f940b0227f8: 126) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0227b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 473 (Thread 0x7f93313f0640 (LWP 2740127) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b43d068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b43d000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b43d000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b43d000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=125) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022790: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0227a0: 0x7f956f09f800, __args=@0x7f940b0227a8: 125) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022790: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0227a8: 125, __args=@0x7f940b0227a8: 125) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022768, args=@0x7f940b0227a8: 125, args=@0x7f940b0227a8: 125) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0227a8: 125, __args=@0x7f940b0227a8: 125) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022760) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 472 (Thread 0x7f9331df1640 (LWP 2740126) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b9d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b9d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b9d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b9d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=124) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022740: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022750: 0x7f956f09f800, __args=@0x7f940b022758: 124) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022740: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022758: 124, __args=@0x7f940b022758: 124) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022718, args=@0x7f940b022758: 124, args=@0x7f940b022758: 124) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022758: 124, __args=@0x7f940b022758: 124) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022710) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 471 (Thread 0x7f93327f2640 (LWP 2740125) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b9a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b9a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b9a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b9a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=123) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0226f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022700: 0x7f956f09f800, __args=@0x7f940b022708: 123) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0226f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022708: 123, __args=@0x7f940b022708: 123) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0226c8, args=@0x7f940b022708: 123, args=@0x7f940b022708: 123) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022708: 123, __args=@0x7f940b022708: 123) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0226c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 470 (Thread 0x7f93331f3640 (LWP 2740124) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b9768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b9700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b9700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b9700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=122) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0226a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0226b0: 0x7f956f09f800, __args=@0x7f940b0226b8: 122) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0226a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0226b8: 122, __args=@0x7f940b0226b8: 122) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022678, args=@0x7f940b0226b8: 122, args=@0x7f940b0226b8: 122) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0226b8: 122, __args=@0x7f940b0226b8: 122) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022670) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 469 (Thread 0x7f9333bf4640 (LWP 2740123) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b9468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b9400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b9400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b9400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=121) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022650: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022660: 0x7f956f09f800, __args=@0x7f940b022668: 121) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022650: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022668: 121, __args=@0x7f940b022668: 121) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022628, args=@0x7f940b022668: 121, args=@0x7f940b022668: 121) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022668: 121, __args=@0x7f940b022668: 121) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022620) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 468 (Thread 0x7f93345f5640 (LWP 2740122) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b9168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b9100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b9100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b9100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=120) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022600: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022610: 0x7f956f09f800, __args=@0x7f940b022618: 120) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022600: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022618: 120, __args=@0x7f940b022618: 120) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0225d8, args=@0x7f940b022618: 120, args=@0x7f940b022618: 120) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022618: 120, __args=@0x7f940b022618: 120) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0225d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 467 (Thread 0x7f9334ff6640 (LWP 2740121) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b8e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b8e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b8e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b8e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=119) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0225b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0225c0: 0x7f956f09f800, __args=@0x7f940b0225c8: 119) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0225b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0225c8: 119, __args=@0x7f940b0225c8: 119) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022588, args=@0x7f940b0225c8: 119, args=@0x7f940b0225c8: 119) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0225c8: 119, __args=@0x7f940b0225c8: 119) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 466 (Thread 0x7f93359f7640 (LWP 2740120) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b8b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b8b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b8b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b8b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=118) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022560: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022570: 0x7f956f09f800, __args=@0x7f940b022578: 118) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022560: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022578: 118, __args=@0x7f940b022578: 118) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022538, args=@0x7f940b022578: 118, args=@0x7f940b022578: 118) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022578: 118, __args=@0x7f940b022578: 118) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022530) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 465 (Thread 0x7f9336bf8640 (LWP 2740119) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b8868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b8800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b8800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b8800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=117) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022510: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022520: 0x7f956f09f800, __args=@0x7f940b022528: 117) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022510: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022528: 117, __args=@0x7f940b022528: 117) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0224e8, args=@0x7f940b022528: 117, args=@0x7f940b022528: 117) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022528: 117, __args=@0x7f940b022528: 117) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0224e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 464 (Thread 0x7f93375f9640 (LWP 2740118) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b8568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b8500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b8500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b8500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=116) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0224c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0224d0: 0x7f956f09f800, __args=@0x7f940b0224d8: 116) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0224c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0224d8: 116, __args=@0x7f940b0224d8: 116) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022498, args=@0x7f940b0224d8: 116, args=@0x7f940b0224d8: 116) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0224d8: 116, __args=@0x7f940b0224d8: 116) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022490) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 463 (Thread 0x7f9337ffa640 (LWP 2740117) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b8268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b8200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b8200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b8200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=115) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022470: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022480: 0x7f956f09f800, __args=@0x7f940b022488: 115) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022470: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022488: 115, __args=@0x7f940b022488: 115) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022448, args=@0x7f940b022488: 115, args=@0x7f940b022488: 115) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022488: 115, __args=@0x7f940b022488: 115) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 462 (Thread 0x7f93389fb640 (LWP 2740116) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=114) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022420: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022430: 0x7f956f09f800, __args=@0x7f940b022438: 114) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022420: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022438: 114, __args=@0x7f940b022438: 114) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0223f8, args=@0x7f940b022438: 114, args=@0x7f940b022438: 114) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022438: 114, __args=@0x7f940b022438: 114) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0223f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 461 (Thread 0x7f93393fc640 (LWP 2740115) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=113) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0223d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0223e0: 0x7f956f09f800, __args=@0x7f940b0223e8: 113) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0223d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0223e8: 113, __args=@0x7f940b0223e8: 113) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0223a8, args=@0x7f940b0223e8: 113, args=@0x7f940b0223e8: 113) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0223e8: 113, __args=@0x7f940b0223e8: 113) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0223a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 460 (Thread 0x7f9339dfd640 (LWP 2740114) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=112) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022380: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022390: 0x7f956f09f800, __args=@0x7f940b022398: 112) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022380: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022398: 112, __args=@0x7f940b022398: 112) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022358, args=@0x7f940b022398: 112, args=@0x7f940b022398: 112) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022398: 112, __args=@0x7f940b022398: 112) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022350) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 459 (Thread 0x7f933a9fe640 (LWP 2740113) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=111) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022330: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022340: 0x7f956f09f800, __args=@0x7f940b022348: 111) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022330: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022348: 111, __args=@0x7f940b022348: 111) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022308, args=@0x7f940b022348: 111, args=@0x7f940b022348: 111) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022348: 111, __args=@0x7f940b022348: 111) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 458 (Thread 0x7f933b3ff640 (LWP 2740112) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=110) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0222e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0222f0: 0x7f956f09f800, __args=@0x7f940b0222f8: 110) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0222e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0222f8: 110, __args=@0x7f940b0222f8: 110) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0222b8, args=@0x7f940b0222f8: 110, args=@0x7f940b0222f8: 110) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0222f8: 110, __args=@0x7f940b0222f8: 110) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0222b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 457 (Thread 0x7f933c5ff640 (LWP 2740111) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b3b7068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b3b7000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b3b7000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b3b7000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=109) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022290: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0222a0: 0x7f956f09f800, __args=@0x7f940b0222a8: 109) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022290: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0222a8: 109, __args=@0x7f940b0222a8: 109) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022268, args=@0x7f940b0222a8: 109, args=@0x7f940b0222a8: 109) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0222a8: 109, __args=@0x7f940b0222a8: 109) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 456 (Thread 0x7f933d5ff640 (LWP 2740110) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b336d68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b336d00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b336d00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b336d00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=108) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022240: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022250: 0x7f956f09f800, __args=@0x7f940b022258: 108) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022240: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022258: 108, __args=@0x7f940b022258: 108) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022218, args=@0x7f940b022258: 108, args=@0x7f940b022258: 108) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022258: 108, __args=@0x7f940b022258: 108) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022210) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 455 (Thread 0x7f933e3ff640 (LWP 2740109) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b336a68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b336a00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b336a00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b336a00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=107) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0221f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022200: 0x7f956f09f800, __args=@0x7f940b022208: 107) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0221f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022208: 107, __args=@0x7f940b022208: 107) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0221c8, args=@0x7f940b022208: 107, args=@0x7f940b022208: 107) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022208: 107, __args=@0x7f940b022208: 107) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0221c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 454 (Thread 0x7f933f7fe640 (LWP 2740108) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b336768, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b336700, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b336700, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b336700, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=106) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0221a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0221b0: 0x7f956f09f800, __args=@0x7f940b0221b8: 106) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0221a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0221b8: 106, __args=@0x7f940b0221b8: 106) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022178, args=@0x7f940b0221b8: 106, args=@0x7f940b0221b8: 106) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0221b8: 106, __args=@0x7f940b0221b8: 106) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022170) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 453 (Thread 0x7f93403ff640 (LWP 2740107) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b336468, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b336400, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b336400, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b336400, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=105) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022150: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022160: 0x7f956f09f800, __args=@0x7f940b022168: 105) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022150: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022168: 105, __args=@0x7f940b022168: 105) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022128, args=@0x7f940b022168: 105, args=@0x7f940b022168: 105) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022168: 105, __args=@0x7f940b022168: 105) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022120) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 452 (Thread 0x7f93415ff640 (LWP 2740106) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b336168, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b336100, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b336100, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b336100, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=104) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022100: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022110: 0x7f956f09f800, __args=@0x7f940b022118: 104) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022100: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022118: 104, __args=@0x7f940b022118: 104) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0220d8, args=@0x7f940b022118: 104, args=@0x7f940b022118: 104) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022118: 104, __args=@0x7f940b022118: 104) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0220d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 451 (Thread 0x7f93425fe640 (LWP 2740105) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b335e68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b335e00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b335e00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b335e00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=103) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0220b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0220c0: 0x7f956f09f800, __args=@0x7f940b0220c8: 103) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0220b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0220c8: 103, __args=@0x7f940b0220c8: 103) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022088, args=@0x7f940b0220c8: 103, args=@0x7f940b0220c8: 103) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0220c8: 103, __args=@0x7f940b0220c8: 103) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 450 (Thread 0x7f9342fff640 (LWP 2740104) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b335b68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b335b00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b335b00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b335b00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=102) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022060: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022070: 0x7f956f09f800, __args=@0x7f940b022078: 102) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022060: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022078: 102, __args=@0x7f940b022078: 102) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b022038, args=@0x7f940b022078: 102, args=@0x7f940b022078: 102) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022078: 102, __args=@0x7f940b022078: 102) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b022030) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 449 (Thread 0x7f9343fff640 (LWP 2740103) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b335868, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b335800, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b335800, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b335800, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=101) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022010: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b022020: 0x7f956f09f800, __args=@0x7f940b022028: 101) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b022010: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b022028: 101, __args=@0x7f940b022028: 101) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021fe8, args=@0x7f940b022028: 101, args=@0x7f940b022028: 101) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b022028: 101, __args=@0x7f940b022028: 101) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021fe0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 448 (Thread 0x7f9344fff640 (LWP 2740102) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b335568, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b335500, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b335500, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b335500, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=100) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021fc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021fd0: 0x7f956f09f800, __args=@0x7f940b021fd8: 100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021fc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021fd8: 100, __args=@0x7f940b021fd8: 100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021f98, args=@0x7f940b021fd8: 100, args=@0x7f940b021fd8: 100) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021fd8: 100, __args=@0x7f940b021fd8: 100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021f90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 447 (Thread 0x7f9345fff640 (LWP 2740101) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b335268, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b335200, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b335200, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b335200, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=99) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021f70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021f80: 0x7f956f09f800, __args=@0x7f940b021f88: 99) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021f70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021f88: 99, __args=@0x7f940b021f88: 99) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021f48, args=@0x7f940b021f88: 99, args=@0x7f940b021f88: 99) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021f88: 99, __args=@0x7f940b021f88: 99) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021f40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 446 (Thread 0x7f9346dfe640 (LWP 2740100) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334f68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334f00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b334f00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334f00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=98) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021f20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021f30: 0x7f956f09f800, __args=@0x7f940b021f38: 98) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021f20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021f38: 98, __args=@0x7f940b021f38: 98) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021ef8, args=@0x7f940b021f38: 98, args=@0x7f940b021f38: 98) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021f38: 98, __args=@0x7f940b021f38: 98) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021ef0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 445 (Thread 0x7f93479ff640 (LWP 2740099) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334c68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334c00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b334c00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334c00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=97) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ed0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021ee0: 0x7f956f09f800, __args=@0x7f940b021ee8: 97) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ed0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021ee8: 97, __args=@0x7f940b021ee8: 97) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021ea8, args=@0x7f940b021ee8: 97, args=@0x7f940b021ee8: 97) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021ee8: 97, __args=@0x7f940b021ee8: 97) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021ea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 444 (Thread 0x7f9348dff640 (LWP 2740098) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334968, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334900, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b334900, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334900, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=96) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021e80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021e90: 0x7f956f09f800, __args=@0x7f940b021e98: 96) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021e80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021e98: 96, __args=@0x7f940b021e98: 96) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021e58, args=@0x7f940b021e98: 96, args=@0x7f940b021e98: 96) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021e98: 96, __args=@0x7f940b021e98: 96) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021e50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 443 (Thread 0x7f9349cfd640 (LWP 2740097) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334668, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334600, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b334600, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334600, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=95) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021e30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021e40: 0x7f956f09f800, __args=@0x7f940b021e48: 95) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021e30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021e48: 95, __args=@0x7f940b021e48: 95) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021e08, args=@0x7f940b021e48: 95, args=@0x7f940b021e48: 95) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021e48: 95, __args=@0x7f940b021e48: 95) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021e00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 442 (Thread 0x7f934a8fe640 (LWP 2740096) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334368, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334300, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b334300, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334300, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=94) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021de0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021df0: 0x7f956f09f800, __args=@0x7f940b021df8: 94) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021de0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021df8: 94, __args=@0x7f940b021df8: 94) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021db8, args=@0x7f940b021df8: 94, args=@0x7f940b021df8: 94) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021df8: 94, __args=@0x7f940b021df8: 94) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021db0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 441 (Thread 0x7f934b2ff640 (LWP 2740095) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b334068, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b334000, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b334000, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b334000, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=93) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021d90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021da0: 0x7f956f09f800, __args=@0x7f940b021da8: 93) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021d90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021da8: 93, __args=@0x7f940b021da8: 93) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021d68, args=@0x7f940b021da8: 93, args=@0x7f940b021da8: 93) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021da8: 93, __args=@0x7f940b021da8: 93) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021d60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 440 (Thread 0x7f9425435640 (LWP 2740094) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bbee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bbe80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bbe80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bbe80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=92) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021d40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021d50: 0x7f956f09f800, __args=@0x7f940b021d58: 92) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021d40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021d58: 92, __args=@0x7f940b021d58: 92) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021d18, args=@0x7f940b021d58: 92, args=@0x7f940b021d58: 92) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021d58: 92, __args=@0x7f940b021d58: 92) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021d10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 439 (Thread 0x7f934d7ff640 (LWP 2740093) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bbbe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bbb80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bbb80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bbb80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=91) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021cf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021d00: 0x7f956f09f800, __args=@0x7f940b021d08: 91) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021cf0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021d08: 91, __args=@0x7f940b021d08: 91) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021cc8, args=@0x7f940b021d08: 91, args=@0x7f940b021d08: 91) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021d08: 91, __args=@0x7f940b021d08: 91) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 438 (Thread 0x7f934e7ff640 (LWP 2740092) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bb8e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bb880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bb880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bb880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=90) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ca0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021cb0: 0x7f956f09f800, __args=@0x7f940b021cb8: 90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ca0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021cb8: 90, __args=@0x7f940b021cb8: 90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021c78, args=@0x7f940b021cb8: 90, args=@0x7f940b021cb8: 90) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021cb8: 90, __args=@0x7f940b021cb8: 90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021c70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 437 (Thread 0x7f934f7ff640 (LWP 2740091) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bb5e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bb580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bb580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bb580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=89) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021c50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021c60: 0x7f956f09f800, __args=@0x7f940b021c68: 89) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021c50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021c68: 89, __args=@0x7f940b021c68: 89) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021c28, args=@0x7f940b021c68: 89, args=@0x7f940b021c68: 89) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021c68: 89, __args=@0x7f940b021c68: 89) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021c20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 436 (Thread 0x7f9350bff640 (LWP 2740090) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bb2e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bb280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bb280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bb280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021c00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021c10: 0x7f956f09f800, __args=@0x7f940b021c18: 88) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021c00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021c18: 88, __args=@0x7f940b021c18: 88) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021bd8, args=@0x7f940b021c18: 88, args=@0x7f940b021c18: 88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021c18: 88, __args=@0x7f940b021c18: 88) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021bd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 435 (Thread 0x7f9351bff640 (LWP 2740089) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bafe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2baf80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2baf80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2baf80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=87) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021bb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021bc0: 0x7f956f09f800, __args=@0x7f940b021bc8: 87) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021bb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021bc8: 87, __args=@0x7f940b021bc8: 87) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021b88, args=@0x7f940b021bc8: 87, args=@0x7f940b021bc8: 87) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021bc8: 87, __args=@0x7f940b021bc8: 87) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021b80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 434 (Thread 0x7f9352bff640 (LWP 2740088) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2bace8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2bac80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2bac80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2bac80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=86) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021b60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021b70: 0x7f956f09f800, __args=@0x7f940b021b78: 86) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021b60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021b78: 86, __args=@0x7f940b021b78: 86) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021b38, args=@0x7f940b021b78: 86, args=@0x7f940b021b78: 86) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021b78: 86, __args=@0x7f940b021b78: 86) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021b30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 433 (Thread 0x7f93539ff640 (LWP 2740087) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2ba9e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2ba980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2ba980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2ba980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=85) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021b10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021b20: 0x7f956f09f800, __args=@0x7f940b021b28: 85) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021b10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021b28: 85, __args=@0x7f940b021b28: 85) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021ae8, args=@0x7f940b021b28: 85, args=@0x7f940b021b28: 85) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021b28: 85, __args=@0x7f940b021b28: 85) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021ae0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 432 (Thread 0x7f9354bff640 (LWP 2740086) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2ba6e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2ba680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2ba680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2ba680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=84) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ac0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021ad0: 0x7f956f09f800, __args=@0x7f940b021ad8: 84) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021ac0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021ad8: 84, __args=@0x7f940b021ad8: 84) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021a98, args=@0x7f940b021ad8: 84, args=@0x7f940b021ad8: 84) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021ad8: 84, __args=@0x7f940b021ad8: 84) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021a90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 431 (Thread 0x7f9356bff640 (LWP 2740085) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2ba3e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2ba380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2ba380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2ba380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=83) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021a70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021a80: 0x7f956f09f800, __args=@0x7f940b021a88: 83) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021a70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021a88: 83, __args=@0x7f940b021a88: 83) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021a48, args=@0x7f940b021a88: 83, args=@0x7f940b021a88: 83) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021a88: 83, __args=@0x7f940b021a88: 83) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021a40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 430 (Thread 0x7f9357bff640 (LWP 2740084) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2ba0e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2ba080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2ba080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2ba080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=82) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021a20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021a30: 0x7f956f09f800, __args=@0x7f940b021a38: 82) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021a20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021a38: 82, __args=@0x7f940b021a38: 82) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0219f8, args=@0x7f940b021a38: 82, args=@0x7f940b021a38: 82) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021a38: 82, __args=@0x7f940b021a38: 82) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0219f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 429 (Thread 0x7f9358bff640 (LWP 2740083) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2b9de8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2b9d80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2b9d80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2b9d80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=81) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0219d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0219e0: 0x7f956f09f800, __args=@0x7f940b0219e8: 81) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0219d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0219e8: 81, __args=@0x7f940b0219e8: 81) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0219a8, args=@0x7f940b0219e8: 81, args=@0x7f940b0219e8: 81) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0219e8: 81, __args=@0x7f940b0219e8: 81) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0219a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 428 (Thread 0x7f93599ff640 (LWP 2740082) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2b9ae8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2b9a80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2b9a80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2b9a80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=80) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021980: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021990: 0x7f956f09f800, __args=@0x7f940b021998: 80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021980: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021998: 80, __args=@0x7f940b021998: 80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021958, args=@0x7f940b021998: 80, args=@0x7f940b021998: 80) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021998: 80, __args=@0x7f940b021998: 80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021950) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 427 (Thread 0x7f935b9fe640 (LWP 2740081) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2b97e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2b9780, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2b9780, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2b9780, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=79) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021930: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021940: 0x7f956f09f800, __args=@0x7f940b021948: 79) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021930: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021948: 79, __args=@0x7f940b021948: 79) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021908, args=@0x7f940b021948: 79, args=@0x7f940b021948: 79) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021948: 79, __args=@0x7f940b021948: 79) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 426 (Thread 0x7f935c5ff640 (LWP 2740080) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2b94e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2b9480, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2b9480, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2b9480, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=78) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0218e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0218f0: 0x7f956f09f800, __args=@0x7f940b0218f8: 78) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0218e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0218f8: 78, __args=@0x7f940b0218f8: 78) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0218b8, args=@0x7f940b0218f8: 78, args=@0x7f940b0218f8: 78) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0218f8: 78, __args=@0x7f940b0218f8: 78) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0218b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 425 (Thread 0x7f935d5ff640 (LWP 2740079) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2b91e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b2b9180, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b2b9180, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b2b9180, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=77) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021890: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0218a0: 0x7f956f09f800, __args=@0x7f940b0218a8: 77) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021890: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0218a8: 77, __args=@0x7f940b0218a8: 77) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021868, args=@0x7f940b0218a8: 77, args=@0x7f940b0218a8: 77) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0218a8: 77, __args=@0x7f940b0218a8: 77) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021860) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 424 (Thread 0x7f935e3ff640 (LWP 2740078) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b208ee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b208e80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b208e80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b208e80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=76) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021840: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021850: 0x7f956f09f800, __args=@0x7f940b021858: 76) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021840: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021858: 76, __args=@0x7f940b021858: 76) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021818, args=@0x7f940b021858: 76, args=@0x7f940b021858: 76) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021858: 76, __args=@0x7f940b021858: 76) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021810) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 423 (Thread 0x7f935fdff640 (LWP 2740077) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b208be8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b208b80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b208b80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b208b80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=75) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0217f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021800: 0x7f956f09f800, __args=@0x7f940b021808: 75) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0217f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021808: 75, __args=@0x7f940b021808: 75) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0217c8, args=@0x7f940b021808: 75, args=@0x7f940b021808: 75) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021808: 75, __args=@0x7f940b021808: 75) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0217c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 422 (Thread 0x7f9360dfe640 (LWP 2740076) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2088e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b208880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b208880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b208880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=74) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0217a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0217b0: 0x7f956f09f800, __args=@0x7f940b0217b8: 74) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0217a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0217b8: 74, __args=@0x7f940b0217b8: 74) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021778, args=@0x7f940b0217b8: 74, args=@0x7f940b0217b8: 74) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0217b8: 74, __args=@0x7f940b0217b8: 74) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021770) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 421 (Thread 0x7f93617ff640 (LWP 2740075) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2085e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b208580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b208580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b208580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=73) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021750: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021760: 0x7f956f09f800, __args=@0x7f940b021768: 73) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021750: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021768: 73, __args=@0x7f940b021768: 73) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021728, args=@0x7f940b021768: 73, args=@0x7f940b021768: 73) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021768: 73, __args=@0x7f940b021768: 73) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 420 (Thread 0x7f9362bff640 (LWP 2740074) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2082e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b208280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b208280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b208280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=72) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021700: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021710: 0x7f956f09f800, __args=@0x7f940b021718: 72) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021700: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021718: 72, __args=@0x7f940b021718: 72) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0216d8, args=@0x7f940b021718: 72, args=@0x7f940b021718: 72) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021718: 72, __args=@0x7f940b021718: 72) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0216d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 419 (Thread 0x7f93639fe640 (LWP 2740073) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b207fe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207f80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b207f80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207f80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=71) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0216b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0216c0: 0x7f956f09f800, __args=@0x7f940b0216c8: 71) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0216b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0216c8: 71, __args=@0x7f940b0216c8: 71) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021688, args=@0x7f940b0216c8: 71, args=@0x7f940b0216c8: 71) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0216c8: 71, __args=@0x7f940b0216c8: 71) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 418 (Thread 0x7f93643ff640 (LWP 2740072) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b207ce8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207c80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b207c80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207c80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=70) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021660: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021670: 0x7f956f09f800, __args=@0x7f940b021678: 70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021660: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021678: 70, __args=@0x7f940b021678: 70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021638, args=@0x7f940b021678: 70, args=@0x7f940b021678: 70) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021678: 70, __args=@0x7f940b021678: 70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021630) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 417 (Thread 0x7f9365bfa640 (LWP 2740071) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2079e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b207980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=69) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021610: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021620: 0x7f956f09f800, __args=@0x7f940b021628: 69) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021610: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021628: 69, __args=@0x7f940b021628: 69) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0215e8, args=@0x7f940b021628: 69, args=@0x7f940b021628: 69) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021628: 69, __args=@0x7f940b021628: 69) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0215e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 416 (Thread 0x7f93665fb640 (LWP 2740070) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af82ae08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af82ae00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 415 (Thread 0x7f9366ffc640 (LWP 2740069) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2076e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b207680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0215c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0215d0: 0x7f956f09f800, __args=@0x7f940b0215d8: 68) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0215c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0215d8: 68, __args=@0x7f940b0215d8: 68) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021598, args=@0x7f940b0215d8: 68, args=@0x7f940b0215d8: 68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0215d8: 68, __args=@0x7f940b0215d8: 68) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021590) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 414 (Thread 0x7f93685fc640 (LWP 2740068) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af82ada8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af82ada0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 413 (Thread 0x7f93691fd640 (LWP 2740067) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2073e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b207380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=67) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021570: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021580: 0x7f956f09f800, __args=@0x7f940b021588: 67) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021570: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021588: 67, __args=@0x7f940b021588: 67) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021548, args=@0x7f940b021588: 67, args=@0x7f940b021588: 67) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021588: 67, __args=@0x7f940b021588: 67) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 412 (Thread 0x7f9369bfe640 (LWP 2740066) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2070e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b207080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b207080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b207080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=66) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021520: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021530: 0x7f956f09f800, __args=@0x7f940b021538: 66) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021520: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021538: 66, __args=@0x7f940b021538: 66) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0214f8, args=@0x7f940b021538: 66, args=@0x7f940b021538: 66) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021538: 66, __args=@0x7f940b021538: 66) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0214f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 411 (Thread 0x7f936b5ff640 (LWP 2740065) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b206de8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b206d80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b206d80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b206d80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=65) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0214d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0214e0: 0x7f956f09f800, __args=@0x7f940b0214e8: 65) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0214d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0214e8: 65, __args=@0x7f940b0214e8: 65) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0214a8, args=@0x7f940b0214e8: 65, args=@0x7f940b0214e8: 65) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0214e8: 65, __args=@0x7f940b0214e8: 65) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0214a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 410 (Thread 0x7f936abfe640 (LWP 2740064) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af82ad48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af82ad40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 409 (Thread 0x7f936cbfe640 (LWP 2740063) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b206ae8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b206a80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b206a80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b206a80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=64) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021480: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021490: 0x7f956f09f800, __args=@0x7f940b021498: 64) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021480: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021498: 64, __args=@0x7f940b021498: 64) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021458, args=@0x7f940b021498: 64, args=@0x7f940b021498: 64) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021498: 64, __args=@0x7f940b021498: 64) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021450) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 408 (Thread 0x7f936d9ff640 (LWP 2740062) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2067e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b206780, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b206780, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b206780, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=63) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021430: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021440: 0x7f956f09f800, __args=@0x7f940b021448: 63) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021430: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021448: 63, __args=@0x7f940b021448: 63) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021408, args=@0x7f940b021448: 63, args=@0x7f940b021448: 63) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021448: 63, __args=@0x7f940b021448: 63) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 407 (Thread 0x7f936e9ff640 (LWP 2740061) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2064e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b206480, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b206480, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b206480, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=62) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0213e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0213f0: 0x7f956f09f800, __args=@0x7f940b0213f8: 62) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0213e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0213f8: 62, __args=@0x7f940b0213f8: 62) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0213b8, args=@0x7f940b0213f8: 62, args=@0x7f940b0213f8: 62) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0213f8: 62, __args=@0x7f940b0213f8: 62) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0213b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 406 (Thread 0x7f936f7fe640 (LWP 2740060) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af82ace8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af82ace0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 405 (Thread 0x7f93705ff640 (LWP 2740059) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b2061e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b206180, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b206180, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b206180, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=61) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021390: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0213a0: 0x7f956f09f800, __args=@0x7f940b0213a8: 61) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021390: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0213a8: 61, __args=@0x7f940b0213a8: 61) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021368, args=@0x7f940b0213a8: 61, args=@0x7f940b0213a8: 61) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0213a8: 61, __args=@0x7f940b0213a8: 61) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 404 (Thread 0x7f93713fd640 (LWP 2740058) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af82ac88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af82ac80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 403 (Thread 0x7f9371ffe640 (LWP 2740057) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b185ee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b185e80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b185e80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b185e80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=60) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021340: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021350: 0x7f956f09f800, __args=@0x7f940b021358: 60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021340: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021358: 60, __args=@0x7f940b021358: 60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021318, args=@0x7f940b021358: 60, args=@0x7f940b021358: 60) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021358: 60, __args=@0x7f940b021358: 60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021310) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 402 (Thread 0x7f9372ffe640 (LWP 2740056) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b185be8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b185b80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b185b80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b185b80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=59) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0212f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021300: 0x7f956f09f800, __args=@0x7f940b021308: 59) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0212f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021308: 59, __args=@0x7f940b021308: 59) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0212c8, args=@0x7f940b021308: 59, args=@0x7f940b021308: 59) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021308: 59, __args=@0x7f940b021308: 59) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0212c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 401 (Thread 0x7f93739ff640 (LWP 2740055) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1858e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b185880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b185880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b185880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=58) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0212a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0212b0: 0x7f956f09f800, __args=@0x7f940b0212b8: 58) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0212a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0212b8: 58, __args=@0x7f940b0212b8: 58) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021278, args=@0x7f940b0212b8: 58, args=@0x7f940b0212b8: 58) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0212b8: 58, __args=@0x7f940b0212b8: 58) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021270) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 400 (Thread 0x7f93749ff640 (LWP 2740054) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1855e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b185580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b185580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b185580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=57) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021250: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021260: 0x7f956f09f800, __args=@0x7f940b021268: 57) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021250: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021268: 57, __args=@0x7f940b021268: 57) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021228, args=@0x7f940b021268: 57, args=@0x7f940b021268: 57) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021268: 57, __args=@0x7f940b021268: 57) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 399 (Thread 0x7f93759ff640 (LWP 2740053) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1852e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b185280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b185280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b185280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=56) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021200: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021210: 0x7f956f09f800, __args=@0x7f940b021218: 56) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021200: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021218: 56, __args=@0x7f940b021218: 56) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0211d8, args=@0x7f940b021218: 56, args=@0x7f940b021218: 56) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021218: 56, __args=@0x7f940b021218: 56) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0211d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 398 (Thread 0x7f93769ff640 (LWP 2740052) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b184fe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184f80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b184f80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184f80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=55) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0211b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0211c0: 0x7f956f09f800, __args=@0x7f940b0211c8: 55) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0211b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0211c8: 55, __args=@0x7f940b0211c8: 55) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021188, args=@0x7f940b0211c8: 55, args=@0x7f940b0211c8: 55) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0211c8: 55, __args=@0x7f940b0211c8: 55) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 397 (Thread 0x7f93777fe640 (LWP 2740051) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b184ce8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184c80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b184c80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184c80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=54) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021160: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021170: 0x7f956f09f800, __args=@0x7f940b021178: 54) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021160: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021178: 54, __args=@0x7f940b021178: 54) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021138, args=@0x7f940b021178: 54, args=@0x7f940b021178: 54) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021178: 54, __args=@0x7f940b021178: 54) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021130) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 396 (Thread 0x7f93783ff640 (LWP 2740050) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1849e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b184980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=53) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021110: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021120: 0x7f956f09f800, __args=@0x7f940b021128: 53) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021110: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021128: 53, __args=@0x7f940b021128: 53) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0210e8, args=@0x7f940b021128: 53, args=@0x7f940b021128: 53) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021128: 53, __args=@0x7f940b021128: 53) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0210e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 395 (Thread 0x7f93793ff640 (LWP 2740049) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1846e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b184680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=52) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0210c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0210d0: 0x7f956f09f800, __args=@0x7f940b0210d8: 52) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0210c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0210d8: 52, __args=@0x7f940b0210d8: 52) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021098, args=@0x7f940b0210d8: 52, args=@0x7f940b0210d8: 52) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0210d8: 52, __args=@0x7f940b0210d8: 52) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021090) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 394 (Thread 0x7f937a3ff640 (LWP 2740048) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1843e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b184380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=51) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021070: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021080: 0x7f956f09f800, __args=@0x7f940b021088: 51) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021070: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021088: 51, __args=@0x7f940b021088: 51) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b021048, args=@0x7f940b021088: 51, args=@0x7f940b021088: 51) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021088: 51, __args=@0x7f940b021088: 51) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b021040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 393 (Thread 0x7f937b7ff640 (LWP 2740047) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1840e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b184080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b184080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b184080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=50) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021020: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b021030: 0x7f956f09f800, __args=@0x7f940b021038: 50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b021020: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b021038: 50, __args=@0x7f940b021038: 50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020ff8, args=@0x7f940b021038: 50, args=@0x7f940b021038: 50) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b021038: 50, __args=@0x7f940b021038: 50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020ff0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 392 (Thread 0x7f937c5ff640 (LWP 2740046) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b183de8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b183d80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b183d80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b183d80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=49) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020fd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020fe0: 0x7f956f09f800, __args=@0x7f940b020fe8: 49) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020fd0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020fe8: 49, __args=@0x7f940b020fe8: 49) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020fa8, args=@0x7f940b020fe8: 49, args=@0x7f940b020fe8: 49) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020fe8: 49, __args=@0x7f940b020fe8: 49) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020fa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 391 (Thread 0x7f937d5ff640 (LWP 2740045) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b183ae8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b183a80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b183a80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b183a80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020f80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020f90: 0x7f956f09f800, __args=@0x7f940b020f98: 48) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020f80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020f98: 48, __args=@0x7f940b020f98: 48) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020f58, args=@0x7f940b020f98: 48, args=@0x7f940b020f98: 48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020f98: 48, __args=@0x7f940b020f98: 48) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020f50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 390 (Thread 0x7f937e3fe640 (LWP 2740044) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1837e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b183780, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b183780, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b183780, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=47) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020f30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020f40: 0x7f956f09f800, __args=@0x7f940b020f48: 47) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020f30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020f48: 47, __args=@0x7f940b020f48: 47) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020f08, args=@0x7f940b020f48: 47, args=@0x7f940b020f48: 47) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020f48: 47, __args=@0x7f940b020f48: 47) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020f00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 389 (Thread 0x7f937f1ff640 (LWP 2740043) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1834e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b183480, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b183480, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b183480, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=46) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020ee0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020ef0: 0x7f956f09f800, __args=@0x7f940b020ef8: 46) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020ee0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020ef8: 46, __args=@0x7f940b020ef8: 46) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020eb8, args=@0x7f940b020ef8: 46, args=@0x7f940b020ef8: 46) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020ef8: 46, __args=@0x7f940b020ef8: 46) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020eb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 388 (Thread 0x7f93801ff640 (LWP 2740042) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1831e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b183180, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b183180, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b183180, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=45) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020e90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020ea0: 0x7f956f09f800, __args=@0x7f940b020ea8: 45) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020e90: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020ea8: 45, __args=@0x7f940b020ea8: 45) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020e68, args=@0x7f940b020ea8: 45, args=@0x7f940b020ea8: 45) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020ea8: 45, __args=@0x7f940b020ea8: 45) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020e60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 387 (Thread 0x7f93811ff640 (LWP 2740041) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b102ee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b102e80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b102e80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b102e80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=44) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020e40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020e50: 0x7f956f09f800, __args=@0x7f940b020e58: 44) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020e40: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020e58: 44, __args=@0x7f940b020e58: 44) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020e18, args=@0x7f940b020e58: 44, args=@0x7f940b020e58: 44) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020e58: 44, __args=@0x7f940b020e58: 44) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020e10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 386 (Thread 0x7f93825ff640 (LWP 2740040) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b102be8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b102b80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b102b80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b102b80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=43) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020df0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020e00: 0x7f956f09f800, __args=@0x7f940b020e08: 43) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020df0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020e08: 43, __args=@0x7f940b020e08: 43) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020dc8, args=@0x7f940b020e08: 43, args=@0x7f940b020e08: 43) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020e08: 43, __args=@0x7f940b020e08: 43) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020dc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 385 (Thread 0x7f93835ff640 (LWP 2740039) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1028e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b102880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b102880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b102880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=42) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020da0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020db0: 0x7f956f09f800, __args=@0x7f940b020db8: 42) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020da0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020db8: 42, __args=@0x7f940b020db8: 42) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020d78, args=@0x7f940b020db8: 42, args=@0x7f940b020db8: 42) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020db8: 42, __args=@0x7f940b020db8: 42) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020d70) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 384 (Thread 0x7f93843ff640 (LWP 2740038) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1025e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b102580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b102580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b102580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=41) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020d50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020d60: 0x7f956f09f800, __args=@0x7f940b020d68: 41) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020d50: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020d68: 41, __args=@0x7f940b020d68: 41) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020d28, args=@0x7f940b020d68: 41, args=@0x7f940b020d68: 41) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020d68: 41, __args=@0x7f940b020d68: 41) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020d20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 383 (Thread 0x7f9385dff640 (LWP 2740037) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1022e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b102280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b102280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b102280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=40) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020d00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020d10: 0x7f956f09f800, __args=@0x7f940b020d18: 40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020d00: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020d18: 40, __args=@0x7f940b020d18: 40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020cd8, args=@0x7f940b020d18: 40, args=@0x7f940b020d18: 40) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020d18: 40, __args=@0x7f940b020d18: 40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020cd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 382 (Thread 0x7f9386dff640 (LWP 2740036) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b101fe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101f80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b101f80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101f80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=39) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020cb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020cc0: 0x7f956f09f800, __args=@0x7f940b020cc8: 39) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020cb0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020cc8: 39, __args=@0x7f940b020cc8: 39) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020c88, args=@0x7f940b020cc8: 39, args=@0x7f940b020cc8: 39) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020cc8: 39, __args=@0x7f940b020cc8: 39) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020c80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 381 (Thread 0x7f9387dff640 (LWP 2740035) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b101ce8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101c80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b101c80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101c80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=38) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020c60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020c70: 0x7f956f09f800, __args=@0x7f940b020c78: 38) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020c60: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020c78: 38, __args=@0x7f940b020c78: 38) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020c38, args=@0x7f940b020c78: 38, args=@0x7f940b020c78: 38) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020c78: 38, __args=@0x7f940b020c78: 38) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020c30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 380 (Thread 0x7f9388dff640 (LWP 2740034) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1019e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b101980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=37) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020c10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020c20: 0x7f956f09f800, __args=@0x7f940b020c28: 37) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020c10: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020c28: 37, __args=@0x7f940b020c28: 37) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020be8, args=@0x7f940b020c28: 37, args=@0x7f940b020c28: 37) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020c28: 37, __args=@0x7f940b020c28: 37) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020be0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 379 (Thread 0x7f9389dff640 (LWP 2740033) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1016e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b101680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=36) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020bc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020bd0: 0x7f956f09f800, __args=@0x7f940b020bd8: 36) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020bc0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020bd8: 36, __args=@0x7f940b020bd8: 36) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020b98, args=@0x7f940b020bd8: 36, args=@0x7f940b020bd8: 36) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020bd8: 36, __args=@0x7f940b020bd8: 36) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020b90) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 378 (Thread 0x7f938adff640 (LWP 2740032) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1013e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b101380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=35) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020b70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020b80: 0x7f956f09f800, __args=@0x7f940b020b88: 35) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020b70: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020b88: 35, __args=@0x7f940b020b88: 35) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020b48, args=@0x7f940b020b88: 35, args=@0x7f940b020b88: 35) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020b88: 35, __args=@0x7f940b020b88: 35) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 377 (Thread 0x7f938bcff640 (LWP 2740031) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1010e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b101080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b101080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b101080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=34) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020b20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020b30: 0x7f956f09f800, __args=@0x7f940b020b38: 34) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020b20: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020b38: 34, __args=@0x7f940b020b38: 34) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020af8, args=@0x7f940b020b38: 34, args=@0x7f940b020b38: 34) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020b38: 34, __args=@0x7f940b020b38: 34) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020af0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 376 (Thread 0x7f938d3ff640 (LWP 2740030) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b100de8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b100d80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b100d80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b100d80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=33) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020ad0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020ae0: 0x7f956f09f800, __args=@0x7f940b020ae8: 33) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020ad0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020ae8: 33, __args=@0x7f940b020ae8: 33) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020aa8, args=@0x7f940b020ae8: 33, args=@0x7f940b020ae8: 33) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020ae8: 33, __args=@0x7f940b020ae8: 33) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020aa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 375 (Thread 0x7f938e3ff640 (LWP 2740029) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b100ae8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b100a80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b100a80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b100a80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=32) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020a80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020a90: 0x7f956f09f800, __args=@0x7f940b020a98: 32) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020a80: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020a98: 32, __args=@0x7f940b020a98: 32) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020a58, args=@0x7f940b020a98: 32, args=@0x7f940b020a98: 32) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020a98: 32, __args=@0x7f940b020a98: 32) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020a50) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 374 (Thread 0x7f938f1ff640 (LWP 2740028) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1007e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b100780, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b100780, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b100780, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=31) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020a30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020a40: 0x7f956f09f800, __args=@0x7f940b020a48: 31) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020a30: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020a48: 31, __args=@0x7f940b020a48: 31) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020a08, args=@0x7f940b020a48: 31, args=@0x7f940b020a48: 31) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020a48: 31, __args=@0x7f940b020a48: 31) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 373 (Thread 0x7f93905ff640 (LWP 2740027) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1004e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b100480, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b100480, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b100480, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=30) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0209e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0209f0: 0x7f956f09f800, __args=@0x7f940b0209f8: 30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0209e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0209f8: 30, __args=@0x7f940b0209f8: 30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0209b8, args=@0x7f940b0209f8: 30, args=@0x7f940b0209f8: 30) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0209f8: 30, __args=@0x7f940b0209f8: 30) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0209b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 372 (Thread 0x7f93915ff640 (LWP 2740026) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b1001e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b100180, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b100180, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b100180, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=29) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020990: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0209a0: 0x7f956f09f800, __args=@0x7f940b0209a8: 29) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020990: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0209a8: 29, __args=@0x7f940b0209a8: 29) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020968, args=@0x7f940b0209a8: 29, args=@0x7f940b0209a8: 29) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0209a8: 29, __args=@0x7f940b0209a8: 29) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020960) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 371 (Thread 0x7f93923ff640 (LWP 2740025) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b077ee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b077e80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b077e80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b077e80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020940: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020950: 0x7f956f09f800, __args=@0x7f940b020958: 28) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020940: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020958: 28, __args=@0x7f940b020958: 28) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020918, args=@0x7f940b020958: 28, args=@0x7f940b020958: 28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020958: 28, __args=@0x7f940b020958: 28) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020910) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 370 (Thread 0x7f93935ff640 (LWP 2740024) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b077be8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b077b80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b077b80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b077b80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=27) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0208f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020900: 0x7f956f09f800, __args=@0x7f940b020908: 27) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0208f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020908: 27, __args=@0x7f940b020908: 27) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0208c8, args=@0x7f940b020908: 27, args=@0x7f940b020908: 27) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020908: 27, __args=@0x7f940b020908: 27) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0208c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 369 (Thread 0x7f93945ff640 (LWP 2740023) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0778e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b077880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b077880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b077880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=26) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0208a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0208b0: 0x7f956f09f800, __args=@0x7f940b0208b8: 26) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0208a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0208b8: 26, __args=@0x7f940b0208b8: 26) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020878, args=@0x7f940b0208b8: 26, args=@0x7f940b0208b8: 26) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0208b8: 26, __args=@0x7f940b0208b8: 26) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020870) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 368 (Thread 0x7f93955ff640 (LWP 2740022) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0775e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b077580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b077580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b077580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=25) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020850: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020860: 0x7f956f09f800, __args=@0x7f940b020868: 25) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020850: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020868: 25, __args=@0x7f940b020868: 25) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020828, args=@0x7f940b020868: 25, args=@0x7f940b020868: 25) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020868: 25, __args=@0x7f940b020868: 25) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020820) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 367 (Thread 0x7f93965ff640 (LWP 2740021) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0772e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b077280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b077280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b077280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=24) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020800: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020810: 0x7f956f09f800, __args=@0x7f940b020818: 24) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020800: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020818: 24, __args=@0x7f940b020818: 24) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0207d8, args=@0x7f940b020818: 24, args=@0x7f940b020818: 24) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020818: 24, __args=@0x7f940b020818: 24) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0207d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 366 (Thread 0x7f93973ff640 (LWP 2740020) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b076fe8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076f80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b076f80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076f80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=23) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0207b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0207c0: 0x7f956f09f800, __args=@0x7f940b0207c8: 23) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0207b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0207c8: 23, __args=@0x7f940b0207c8: 23) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020788, args=@0x7f940b0207c8: 23, args=@0x7f940b0207c8: 23) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0207c8: 23, __args=@0x7f940b0207c8: 23) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 365 (Thread 0x7f93985ff640 (LWP 2740019) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b076ce8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076c80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b076c80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076c80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=22) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020760: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020770: 0x7f956f09f800, __args=@0x7f940b020778: 22) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020760: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020778: 22, __args=@0x7f940b020778: 22) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020738, args=@0x7f940b020778: 22, args=@0x7f940b020778: 22) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020778: 22, __args=@0x7f940b020778: 22) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020730) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 364 (Thread 0x7f93995ff640 (LWP 2740018) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0769e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b076980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=21) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020710: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020720: 0x7f956f09f800, __args=@0x7f940b020728: 21) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020710: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020728: 21, __args=@0x7f940b020728: 21) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0206e8, args=@0x7f940b020728: 21, args=@0x7f940b020728: 21) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020728: 21, __args=@0x7f940b020728: 21) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0206e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 363 (Thread 0x7f939a5ff640 (LWP 2740017) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0766e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b076680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=20) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0206c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0206d0: 0x7f956f09f800, __args=@0x7f940b0206d8: 20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0206c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0206d8: 20, __args=@0x7f940b0206d8: 20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020698, args=@0x7f940b0206d8: 20, args=@0x7f940b0206d8: 20) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0206d8: 20, __args=@0x7f940b0206d8: 20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020690) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 362 (Thread 0x7f939b4ff640 (LWP 2740016) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0763e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b076380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=19) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020670: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020680: 0x7f956f09f800, __args=@0x7f940b020688: 19) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020670: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020688: 19, __args=@0x7f940b020688: 19) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020648, args=@0x7f940b020688: 19, args=@0x7f940b020688: 19) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020688: 19, __args=@0x7f940b020688: 19) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 361 (Thread 0x7f939cbff640 (LWP 2740015) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0760e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b076080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b076080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b076080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=18) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020620: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020630: 0x7f956f09f800, __args=@0x7f940b020638: 18) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020620: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020638: 18, __args=@0x7f940b020638: 18) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0205f8, args=@0x7f940b020638: 18, args=@0x7f940b020638: 18) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020638: 18, __args=@0x7f940b020638: 18) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0205f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 360 (Thread 0x7f939d9ff640 (LWP 2740014) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b075de8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b075d80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b075d80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b075d80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=17) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0205d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0205e0: 0x7f956f09f800, __args=@0x7f940b0205e8: 17) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0205d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0205e8: 17, __args=@0x7f940b0205e8: 17) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0205a8, args=@0x7f940b0205e8: 17, args=@0x7f940b0205e8: 17) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0205e8: 17, __args=@0x7f940b0205e8: 17) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0205a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 359 (Thread 0x7f939ebff640 (LWP 2740013) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b075ae8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b075a80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b075a80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b075a80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=16) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020580: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020590: 0x7f956f09f800, __args=@0x7f940b020598: 16) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020580: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020598: 16, __args=@0x7f940b020598: 16) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020558, args=@0x7f940b020598: 16, args=@0x7f940b020598: 16) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020598: 16, __args=@0x7f940b020598: 16) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020550) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 358 (Thread 0x7f939fbff640 (LWP 2740012) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0757e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b075780, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b075780, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b075780, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=15) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020530: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020540: 0x7f956f09f800, __args=@0x7f940b020548: 15) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020530: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020548: 15, __args=@0x7f940b020548: 15) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020508, args=@0x7f940b020548: 15, args=@0x7f940b020548: 15) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020548: 15, __args=@0x7f940b020548: 15) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 357 (Thread 0x7f93a0ffe640 (LWP 2740011) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0754e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b075480, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b075480, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b075480, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=14) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0204e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0204f0: 0x7f956f09f800, __args=@0x7f940b0204f8: 14) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0204e0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0204f8: 14, __args=@0x7f940b0204f8: 14) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0204b8, args=@0x7f940b0204f8: 14, args=@0x7f940b0204f8: 14) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0204f8: 14, __args=@0x7f940b0204f8: 14) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0204b0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 356 (Thread 0x7f93a19ff640 (LWP 2740010) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f940b0751e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f940b075180, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f940b075180, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f940b075180, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=13) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020490: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0204a0: 0x7f956f09f800, __args=@0x7f940b0204a8: 13) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020490: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0204a8: 13, __args=@0x7f940b0204a8: 13) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020468, args=@0x7f940b0204a8: 13, args=@0x7f940b0204a8: 13) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0204a8: 13, __args=@0x7f940b0204a8: 13) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020460) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 355 (Thread 0x7f93a2dff640 (LWP 2740009) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f9493790ee8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f9493790e80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f9493790e80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f9493790e80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=12) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020440: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020450: 0x7f956f09f800, __args=@0x7f940b020458: 12) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020440: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020458: 12, __args=@0x7f940b020458: 12) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020418, args=@0x7f940b020458: 12, args=@0x7f940b020458: 12) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020458: 12, __args=@0x7f940b020458: 12) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020410) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 354 (Thread 0x7f93a3bfe640 (LWP 2740008) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f9493790be8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f9493790b80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f9493790b80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f9493790b80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=11) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0203f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020400: 0x7f956f09f800, __args=@0x7f940b020408: 11) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0203f0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020408: 11, __args=@0x7f940b020408: 11) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0203c8, args=@0x7f940b020408: 11, args=@0x7f940b020408: 11) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020408: 11, __args=@0x7f940b020408: 11) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0203c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 353 (Thread 0x7f93a49ff640 (LWP 2740007) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f94937908e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f9493790880, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f9493790880, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f9493790880, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=10) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0203a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0203b0: 0x7f956f09f800, __args=@0x7f940b0203b8: 10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0203a0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0203b8: 10, __args=@0x7f940b0203b8: 10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020378, args=@0x7f940b0203b8: 10, args=@0x7f940b0203b8: 10) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0203b8: 10, __args=@0x7f940b0203b8: 10) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020370) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 352 (Thread 0x7f93a5bff640 (LWP 2740006) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f94937905e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f9493790580, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f9493790580, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f9493790580, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=9) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020350: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020360: 0x7f956f09f800, __args=@0x7f940b020368: 9) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020350: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020368: 9, __args=@0x7f940b020368: 9) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020328, args=@0x7f940b020368: 9, args=@0x7f940b020368: 9) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020368: 9, __args=@0x7f940b020368: 9) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 351 (Thread 0x7f93a69fe640 (LWP 2740005) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f94937902e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f9493790280, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f9493790280, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f9493790280, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020300: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020310: 0x7f956f09f800, __args=@0x7f940b020318: 8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020300: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020318: 8, __args=@0x7f940b020318: 8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0202d8, args=@0x7f940b020318: 8, args=@0x7f940b020318: 8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020318: 8, __args=@0x7f940b020318: 8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0202d0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 350 (Thread 0x7f93a75ff640 (LWP 2740004) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c25949d41 in std::__1::condition_variable::wait(std::__1::unique_lock&, DB::PipelineExecutorContext::wait()::$_0) (this=0x7f93a3ca24c0, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c25949ba2 in DB::PipelineExecutorContext::wait (this=0x7f93a3ca2440) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Executor/PipelineExecutorContext.cpp:114 -#5 0x0000558c259417d4 in DB::PipelineExecutor::wait (this=0x7f93a3ca2400) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Executor/PipelineExecutor.cpp:81 -#6 0x0000558c259419c2 in DB::PipelineExecutor::execute (this=0x7f93a3ca2400, result_handler=...) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Executor/PipelineExecutor.cpp:108 -#7 0x0000558c2595ee87 in DB::QueryExecutor::execute (this=0x7f93a3ca2400) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Executor/QueryExecutor.cpp:29 -#8 0x0000558c258716fc in DB::MPPTask::runImpl (this=0x7f9392664800) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Mpp/MPPTask.cpp:617 -#9 0x0000558c258768dd in DB::MPPTask::run()::$_0::operator()() const (this=0x7f939265ab28) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Mpp/MPPTask.cpp:234 -#10 0x0000558c258768b5 in std::__1::__invoke[abi:ue170006](DB::MPPTask::run()::$_0&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#11 0x0000558c25876875 in std::__1::__invoke_void_return_wrapper::__call[abi:ue170006](DB::MPPTask::run()::$_0&) (__args=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:415 -#12 0x0000558c2587684d in std::__1::__function::__alloc_func, void ()>::operator()[abi:ue170006]() (this=0x7f939265ab28) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:192 -#13 0x0000558c25875b49 in std::__1::__function::__func, void ()>::operator()() (this=0x7f939265ab20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:363 -#14 0x0000558c1abb48d2 in std::__1::__function::__value_func::operator()[abi:ue170006]() const (this=0x7f939265ab20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:517 -#15 0x0000558c1aba7925 in std::__1::function::operator()() const (this=0x7f939265ab20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:1168 -#16 0x0000558c238334d5 in std::__1::__invoke[abi:ue170006]>(std::__1::function&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#17 0x0000558c238334b9 in std::__1::__apply_tuple_impl[abi:ue170006], std::__1::tuple<>>(std::__1::function&&, std::__1::tuple<>&&, std::__1::__tuple_indices<>) (__f=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/tuple:1825 -#18 0x0000558c2383348d in std::__1::apply[abi:ue170006], std::__1::tuple<> >(std::__1::function&&, std::__1::tuple<>&&) (__f=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/tuple:1834 -#19 0x0000558c2383343b in DB::wrapInvocable>(bool, std::__1::function&&)::{lambda()#1}::operator()() (this=0x7f939265ab10) at /DATA/disk3/xzx/tiflash/dbms/src/Common/wrapInvocable.h:36 -#20 0x0000558c23833e09 in DB::ExecutableTask>(bool, std::__1::function&&)::{lambda()#1}>::execute() (this=0x7f939265ab00) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ExecutableTask.h:50 -#21 0x0000558c1ac03568 in DB::DynamicThreadPool::executeTask (task=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:124 -#22 0x0000558c1ac031ea in DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=7) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:138 -#23 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0202b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0202c0: 0x7f956f09f800, __args=@0x7f940b0202c8: 7) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#24 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0202b0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0202c8: 7, __args=@0x7f940b0202c8: 7) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#25 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020288, args=@0x7f940b0202c8: 7, args=@0x7f940b0202c8: 7) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#26 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0202c8: 7, __args=@0x7f940b0202c8: 7) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#27 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#28 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#29 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#30 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 349 (Thread 0x7f93a85ff640 (LWP 2740003) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378fce8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378fc80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f949378fc80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378fc80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=6) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020260: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020270: 0x7f956f09f800, __args=@0x7f940b020278: 6) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020260: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020278: 6, __args=@0x7f940b020278: 6) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020238, args=@0x7f940b020278: 6, args=@0x7f940b020278: 6) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020278: 6, __args=@0x7f940b020278: 6) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020230) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 348 (Thread 0x7f93a95ff640 (LWP 2740002) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378f9e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378f980, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f949378f980, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378f980, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=5) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020210: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020220: 0x7f956f09f800, __args=@0x7f940b020228: 5) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020210: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020228: 5, __args=@0x7f940b020228: 5) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0201e8, args=@0x7f940b020228: 5, args=@0x7f940b020228: 5) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020228: 5, __args=@0x7f940b020228: 5) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0201e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 347 (Thread 0x7f93aa6fc640 (LWP 2740001) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378f6e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378f680, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f949378f680, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378f680, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=4) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0201c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0201d0: 0x7f956f09f800, __args=@0x7f940b0201d8: 4) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0201c0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0201d8: 4, __args=@0x7f940b0201d8: 4) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020198, args=@0x7f940b0201d8: 4, args=@0x7f940b0201d8: 4) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0201d8: 4, __args=@0x7f940b0201d8: 4) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020190) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 346 (Thread 0x7f93ab0fd640 (LWP 2740000) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378f3e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378f380, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f949378f380, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378f380, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=3) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020170: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020180: 0x7f956f09f800, __args=@0x7f940b020188: 3) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020170: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020188: 3, __args=@0x7f940b020188: 3) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020148, args=@0x7f940b020188: 3, args=@0x7f940b020188: 3) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020188: 3, __args=@0x7f940b020188: 3) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 345 (Thread 0x7f93ac5fd640 (LWP 2739999) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378f0e8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378f080, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f949378f080, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378f080, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=2) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020120: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020130: 0x7f956f09f800, __args=@0x7f940b020138: 2) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020120: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020138: 2, __args=@0x7f940b020138: 2) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0200f8, args=@0x7f940b020138: 2, args=@0x7f940b020138: 2) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020138: 2, __args=@0x7f940b020138: 2) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0200f0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 344 (Thread 0x7f93acffe640 (LWP 2739998) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378ede8, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378ed80, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f949378ed80, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378ed80, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=1) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0200d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b0200e0: 0x7f956f09f800, __args=@0x7f940b0200e8: 1) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b0200d0: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b0200e8: 1, __args=@0x7f940b0200e8: 1) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b0200a8, args=@0x7f940b0200e8: 1, args=@0x7f940b0200e8: 1) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b0200e8: 1, __args=@0x7f940b0200e8: 1) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b0200a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 343 (Thread 0x7f93ad9ff640 (LWP 2739997) "FixedThread"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ac0e134 in DB::MPMCQueueDetail::WaitingNode::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}) (this=0x7f949378ec68, lock=..., pred=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:68 -#4 DB::MPMCQueue > >::wait > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}>(std::__1::unique_lock&, DB::MPMCQueueDetail::WaitingNode&, DB::MPMCQueue > >::popObj(std::__1::unique_ptr >&, std::__1::chrono::time_point > > const*)::{lambda()#1}, std::__1::chrono::time_point > > const*) (this=0x7f949378ec00, lock=..., head=..., pred=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:296 -#5 DB::MPMCQueue > >::popObj (this=0x7f949378ec00, res=..., deadline=0x0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:313 -#6 0x0000558c1ac03197 in DB::MPMCQueue > >::pop (this=0x7f949378ec00, obj=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/MPMCQueue.h:164 -#7 DB::DynamicThreadPool::fixedWork (this=0x7f956f09f800, index=0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/DynamicThreadPool.cpp:135 -#8 0x0000558c1ac0cec4 in std::__1::__invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020080: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __a0=@0x7f940b020090: 0x7f956f09f800, __args=@0x7f940b020098: 0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#9 0x0000558c1ac0ce05 in std::__1::invoke[abi:ue170006](void (DB::DynamicThreadPool::* const&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=@0x7f940b020080: (void (DB::DynamicThreadPool::*)(class DB::DynamicThreadPool * const, unsigned long)) 0x558c1ac03080 , __args=@0x7f940b020098: 0, __args=@0x7f940b020098: 0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#10 0x0000558c1ac0cd73 in DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}::operator()(DB::DynamicThreadPool*&&, unsigned long&&) const (this=0x7f940b020058, args=@0x7f940b020098: 0, args=@0x7f940b020098: 0) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#11 0x0000558c1ac0cba5 in std::__1::__invoke[abi:ue170006](bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>(void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&) (__f=..., __args=@0x7f940b020098: 0, __args=@0x7f940b020098: 0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#12 0x0000558c1ac0cb6f in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long, 2ul, 3ul>(std::__1::tuple >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long>&, std::__1::__tuple_indices<2ul, 3ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#13 0x0000558c1ac0c8a2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread(bool, std::__1::basic_string, std::__1::allocator >, void (DB::DynamicThreadPool::*&&)(unsigned long), DB::DynamicThreadPool*&&, unsigned long&)::{lambda((auto:1&&)...)#1}, DB::DynamicThreadPool*, unsigned long> >(void*) (__vp=0x7f940b020050) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 342 (Thread 0x7f93babfc640 (LWP 2739996) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b2c3fca8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b2c3fca0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 341 (Thread 0x7f93bbffd640 (LWP 2739995) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b2c3fc48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b2c3fc40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 340 (Thread 0x7f93bc9fe640 (LWP 2739994) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b2c3fbe8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b2c3fbe0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 339 (Thread 0x7f93bd5ff640 (LWP 2739993) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b2c3fb88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b2c3fb80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 338 (Thread 0x7f93c2dff640 (LWP 2739992) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4426688) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4426680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 337 (Thread 0x7f93c3dff640 (LWP 2739991) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4426628) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4426620) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 336 (Thread 0x7f93c4bfe640 (LWP 2739990) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b44265c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b44265c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 335 (Thread 0x7f93c55ff640 (LWP 2739989) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4426568) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4426560) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 334 (Thread 0x7f93c68ff640 (LWP 2739988) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af435fc8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af435fc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 333 (Thread 0x7f93c7c7f640 (LWP 2739987) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af435f68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af435f60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 332 (Thread 0x7f93c93ff640 (LWP 2739986) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af435f08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af435f00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 331 (Thread 0x7f93ca1fe640 (LWP 2739985) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af435ea8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af435ea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 330 (Thread 0x7f93cb67f640 (LWP 2739984) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af435e48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af435e40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 329 (Thread 0x7f93d1dfe640 (LWP 2739983) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ac4207a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ac4207a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 328 (Thread 0x7f93d31ff640 (LWP 2739982) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ac420748) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ac420740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 327 (Thread 0x7f93d63ff640 (LWP 2739981) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00e28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00e20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 326 (Thread 0x7f93d73ff640 (LWP 2739980) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00dc8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00dc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 325 (Thread 0x7f93d83ff640 (LWP 2739979) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00d68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00d60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 324 (Thread 0x7f93d93fd640 (LWP 2739978) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa04ba8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa04ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 323 (Thread 0x7f93d9dfe640 (LWP 2739977) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3b068) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3b060) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 322 (Thread 0x7f93da7ff640 (LWP 2739976) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3b088) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3b080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 321 (Thread 0x7f93db7ff640 (LWP 2739975) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9a04ac8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9a04ac0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 320 (Thread 0x7f93dc7ff640 (LWP 2739974) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9a04a68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9a04a60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 319 (Thread 0x7f93ddfff640 (LWP 2739973) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9401368) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9401360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 318 (Thread 0x7f93defff640 (LWP 2739972) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00d08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00d00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 317 (Thread 0x7f93dffff640 (LWP 2739971) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b3a00648) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b3a00640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 316 (Thread 0x7f93e0dff640 (LWP 2739970) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3bd68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3bd60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 315 (Thread 0x7f93e21ff640 (LWP 2739969) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3bdc8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3bdc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 314 (Thread 0x7f93e2fff640 (LWP 2739968) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9a04a08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9a04a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 313 (Thread 0x7f93e3fff640 (LWP 2739967) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94c4801888) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94c4801880) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 312 (Thread 0x7f93e4dfe640 (LWP 2739966) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94c4801848) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94c4801840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 311 (Thread 0x7f93e5bff640 (LWP 2739965) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4a009a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4a009a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 310 (Thread 0x7f93e6bff640 (LWP 2739964) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b34019a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b34019a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 309 (Thread 0x7f93e7dff640 (LWP 2739963) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b7aebf68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b7aebf60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 308 (Thread 0x7f93e8d7e640 (LWP 2739962) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4a00928) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4a00920) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 307 (Thread 0x7f93e977f640 (LWP 2739961) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9a049a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9a049a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 306 (Thread 0x7f93eabff640 (LWP 2739960) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4c01508) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4c01500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 305 (Thread 0x7f93ebbff640 (LWP 2739959) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00ca8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00ca0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 304 (Thread 0x7f93ecbff640 (LWP 2739958) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b5a00c48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b5a00c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 303 (Thread 0x7f93edefe640 (LWP 2739957) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ca0016c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ca0016c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 302 (Thread 0x7f93eebff640 (LWP 2739956) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3be08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3be00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 301 (Thread 0x7f93efdff640 (LWP 2739955) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3be88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3be80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 300 (Thread 0x7f93f0fff640 (LWP 2739954) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa3bea8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa3bea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 299 (Thread 0x7f93f1fff640 (LWP 2739953) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b3a005e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b3a005e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 298 (Thread 0x7f93f2fff640 (LWP 2739952) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94cdc012a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94cdc012a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 297 (Thread 0x7f93f3fff640 (LWP 2739951) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b0e00468) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b0e00460) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 296 (Thread 0x7f93f4fff640 (LWP 2739950) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94adc007c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94adc007c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 295 (Thread 0x7f93f5fff640 (LWP 2739949) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b0000588) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b0000580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 294 (Thread 0x7f93f6fff640 (LWP 2739948) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213f08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213f00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 293 (Thread 0x7f93f7fff640 (LWP 2739947) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b42005e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b42005e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 292 (Thread 0x7f93f8efe640 (LWP 2739946) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213ea8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213ea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 291 (Thread 0x7f93f98ff640 (LWP 2739945) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b1600728) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b1600720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 290 (Thread 0x7f93fabff640 (LWP 2739944) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213e48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213e40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 289 (Thread 0x7f93fbb7f640 (LWP 2739943) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bfe01368) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bfe01360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 288 (Thread 0x7f93fcbff640 (LWP 2739942) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213de8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213de0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 287 (Thread 0x7f93fe1f8640 (LWP 2739941) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bb3a3e28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bb3a3e20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 286 (Thread 0x7f93fedf9640 (LWP 2739940) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213d88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213d80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 285 (Thread 0x7f93ff7fa640 (LWP 2739939) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bba04448) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bba04440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 284 (Thread 0x7f94001fb640 (LWP 2739938) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213d28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213d20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 283 (Thread 0x7f9400bfc640 (LWP 2739937) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94cc001508) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94cc001500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 282 (Thread 0x7f94015fd640 (LWP 2739936) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213cc8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 281 (Thread 0x7f9401ffe640 (LWP 2739935) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94cae01248) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94cae01240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 280 (Thread 0x7f94029ff640 (LWP 2739934) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213c68) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213c60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 279 (Thread 0x7f9403400640 (LWP 2739933) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94cdc01248) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94cdc01240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 278 (Thread 0x7f9403e01640 (LWP 2739932) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213c08) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213c00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 277 (Thread 0x7f9404802640 (LWP 2739931) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9250168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9250160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 276 (Thread 0x7f9405203640 (LWP 2739930) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213ba8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 275 (Thread 0x7f9405c04640 (LWP 2739929) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ca001188) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ca001180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 274 (Thread 0x7f9406605640 (LWP 2739928) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213b48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 273 (Thread 0x7f9407006640 (LWP 2739927) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ab8014a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ab8014a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 272 (Thread 0x7f9407a07640 (LWP 2739926) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213ae8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213ae0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 271 (Thread 0x7f9408408640 (LWP 2739925) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ac200268) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ac200260) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 270 (Thread 0x7f9408e09640 (LWP 2739924) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213a88) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 269 (Thread 0x7f940980a640 (LWP 2739923) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94adc004c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94adc004c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 268 (Thread 0x7f940a20b640 (LWP 2739922) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213a28) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213a20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 267 (Thread 0x7f940ac0c640 (LWP 2739921) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b4a001a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b4a001a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 266 (Thread 0x7f940c40d640 (LWP 2739920) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62139c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62139c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 265 (Thread 0x7f940ce0e640 (LWP 2739919) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94af8013e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94af8013e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 264 (Thread 0x7f940d80f640 (LWP 2739918) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213968) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213960) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 263 (Thread 0x7f940e210640 (LWP 2739917) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b00001e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b00001e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 262 (Thread 0x7f940ec11640 (LWP 2739916) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213908) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 261 (Thread 0x7f940f612640 (LWP 2739915) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b1e001a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b1e001a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 260 (Thread 0x7f9410013640 (LWP 2739914) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62138a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62138a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 259 (Thread 0x7f9410a14640 (LWP 2739913) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b16004a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b16004a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 258 (Thread 0x7f9411415640 (LWP 2739912) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213848) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 257 (Thread 0x7f9411e16640 (LWP 2739911) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b42002e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b42002e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 256 (Thread 0x7f9412817640 (LWP 2739910) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62137e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62137e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 255 (Thread 0x7f9413218640 (LWP 2739909) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b2c011e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b2c011e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 254 (Thread 0x7f9413c19640 (LWP 2739908) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213788) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 253 (Thread 0x7f941461a640 (LWP 2739907) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b3401228) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b3401220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 252 (Thread 0x7f941501b640 (LWP 2739906) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213728) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 251 (Thread 0x7f9415a1c640 (LWP 2739905) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b8a4e3c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b8a4e3c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 250 (Thread 0x7f941641d640 (LWP 2739904) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62136c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62136c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 249 (Thread 0x7f9416e1e640 (LWP 2739903) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b8a26768) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b8a26760) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 248 (Thread 0x7f941781f640 (LWP 2739902) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213668) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213660) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 247 (Thread 0x7f9418220640 (LWP 2739901) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b52001a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b52001a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 246 (Thread 0x7f9418c21640 (LWP 2739900) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213608) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 245 (Thread 0x7f9419622640 (LWP 2739899) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ba204488) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ba204480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 244 (Thread 0x7f941a023640 (LWP 2739898) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62135a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62135a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 243 (Thread 0x7f941aa24640 (LWP 2739897) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bb3e7528) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bb3e7520) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 242 (Thread 0x7f941b425640 (LWP 2739896) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213548) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 241 (Thread 0x7f941be26640 (LWP 2739895) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bba04068) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bba04060) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 240 (Thread 0x7f941c827640 (LWP 2739894) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62134e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62134e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 239 (Thread 0x7f941d228640 (LWP 2739893) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b7aeb508) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b7aeb500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 238 (Thread 0x7f941dc29640 (LWP 2739892) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213488) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 237 (Thread 0x7f941e62a640 (LWP 2739891) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b8a51c48) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b8a51c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 236 (Thread 0x7f941f02b640 (LWP 2739890) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213428) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213420) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 235 (Thread 0x7f941fa2c640 (LWP 2739889) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b7404488) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b7404480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 234 (Thread 0x7f942042d640 (LWP 2739888) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62133c8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62133c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 233 (Thread 0x7f9420e2e640 (LWP 2739887) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b92505e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b92505e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 232 (Thread 0x7f942182f640 (LWP 2739886) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213368) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 231 (Thread 0x7f9422230640 (LWP 2739885) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213308) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 230 (Thread 0x7f9422c31640 (LWP 2739884) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62132a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62132a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 229 (Thread 0x7f9423632640 (LWP 2739883) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213248) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 228 (Thread 0x7f9424033640 (LWP 2739882) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b62131e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b62131e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 227 (Thread 0x7f9424a34640 (LWP 2739881) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b6213188) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b6213180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 226 (Thread 0x7f9428636640 (LWP 2739831) "grpc_global_tim"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x0000558c288f9ed9 in absl::lts_20211102::synchronization_internal::FutexImpl::WaitUntil (v=0x7f958456e740, val=0, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/futex.h:97 -#2 0x0000558c288f9bfb in absl::lts_20211102::synchronization_internal::Waiter::Wait (this=0x7f958456e740, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/waiter.cc:95 -#3 0x0000558c288f9816 in AbslInternalPerThreadSemWait_lts_20211102 (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.cc:93 -#4 0x0000558c288f6bbd in absl::lts_20211102::synchronization_internal::PerThreadSem::Wait (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.h:112 -#5 0x0000558c288efd56 in absl::lts_20211102::Mutex::DecrementSynchSem (mu=0x558c29567628 , w=0x7f958456e700, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:579 -#6 0x0000558c288f4e7a in absl::lts_20211102::CondVar::WaitCommon (this=0x558c29567630 , mutex=0x558c29567628 , t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2559 -#7 0x0000558c288f5058 in absl::lts_20211102::CondVar::WaitWithDeadline (this=0x558c29567630 , mu=0x558c29567628 , deadline=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2590 -#8 0x0000558c28330047 in gpr_cv_wait (cv=0x558c29567630 , mu=0x558c29567628 , abs_deadline=...) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gpr/sync_abseil.cc:92 -#9 0x0000558c27e19621 in wait_until (next=57318793) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:200 -#10 0x0000558c27e191e9 in timer_main_loop () at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:255 -#11 0x0000558c27e190ef in timer_thread (completed_thread_ptr=0x7f94b6bff000) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:284 -#12 0x0000558c28339f22 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::operator()(void*) const (this=0x7f9428602c87, v=0x7f94b6bff060) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:140 -#13 0x0000558c28339e09 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) (v=0x7f94b6bff060) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:110 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 225 (Thread 0x7f9429037640 (LWP 2739830) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bba04108) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bba04100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 224 (Thread 0x7f9429a38640 (LWP 2739829) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94b9a04168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94b9a04160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 223 (Thread 0x7f942a439640 (LWP 2739828) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94ba204168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94ba204160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 222 (Thread 0x7f942ae3a640 (LWP 2739827) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94baa04168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94baa04160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 221 (Thread 0x7f942b83b640 (LWP 2739826) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94bb204168) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94bb204160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 220 (Thread 0x7f942c23c640 (LWP 2739825) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94937237e8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94937237e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 219 (Thread 0x7f942cc3d640 (LWP 2739824) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f9493723788) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f9493723780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 218 (Thread 0x7f942d63e640 (LWP 2739823) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f9493723708) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f9493723700) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 217 (Thread 0x7f942e03f640 (LWP 2739822) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f94937236a8) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f94937236a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 216 (Thread 0x7f942ea40640 (LWP 2739821) "ThreadPool"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1aba7791 in std::__1::condition_variable::wait::worker(std::__1::__list_iterator)::{lambda()#1}>(std::__1::unique_lock&, DB::ThreadPoolImpl::worker(std::__1::__list_iterator)::{lambda()#1}) (this=0x7f95702f0f58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c1aba72dd in DB::ThreadPoolImpl::worker (this=0x7f95702f0f00, thread_it=...) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:295 -#5 0x0000558c1abaf594 in DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}::operator()() const (this=0x7f9493723648) at /DATA/disk3/xzx/tiflash/dbms/src/Common/UniThreadPool.cpp:167 -#6 0x0000558c1abaf545 in std::__1::__invoke[abi:ue170006]::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c1abaf51d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>(std::__1::tuple >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c1abaf342 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadPoolImpl::scheduleImpl(std::__1::function, long, std::__1::optional, bool)::{lambda()#2}> >(void*) (__vp=0x7f9493723640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 215 (Thread 0x7f942f441640 (LWP 2739820) "segment-sched"): -#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 -#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 -#2 0x00007f95847adc8b in std::__1::this_thread::sleep_for(std::__1::chrono::duration > const&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ab783b2 in std::__1::this_thread::sleep_for[abi:ue170006] >(std::__1::chrono::duration > const&) (__d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/this_thread.h:55 -#4 0x0000558c2331b5e0 in DB::DM::SegmentReadTaskScheduler::schedLoop (this=0x558c28f73ba0 ) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.cpp:324 -#5 0x0000558c23326ac9 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReadTaskScheduler::*&&)(), DB::DM::SegmentReadTaskScheduler*&&) (__f=@0x7f94937235e8: (void (DB::DM::SegmentReadTaskScheduler::*)(class DB::DM::SegmentReadTaskScheduler * const)) 0x558c2331b580 , __a0=@0x7f94937235f8: 0x558c28f73ba0 ) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#6 0x0000558c23326a4e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReadTaskScheduler::*)(), DB::DM::SegmentReadTaskScheduler*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReadTaskScheduler::*)(), DB::DM::SegmentReadTaskScheduler*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#7 0x0000558c23326822 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReadTaskScheduler::*)(), DB::DM::SegmentReadTaskScheduler*> >(void*) (__vp=0x7f94937235e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#8 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#9 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 214 (Thread 0x7f942fe42640 (LWP 2739819) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493721140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493721140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717cc8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717cd8: 0x7f9493721140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 213 (Thread 0x7f9430843640 (LWP 2739818) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94937210c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94937210c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717ca8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717cb8: 0x7f94937210c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717ca0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 212 (Thread 0x7f9431244640 (LWP 2739817) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493721040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493721040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717c88: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717c98: 0x7f9493721040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717c80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 211 (Thread 0x7f9431c45640 (LWP 2739816) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f94fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f94fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717c68: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717c78: 0x7f9493f94fc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717c60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 210 (Thread 0x7f9432646640 (LWP 2739815) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f94f40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f94f40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717c48: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717c58: 0x7f9493f94f40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 209 (Thread 0x7f9433047640 (LWP 2739814) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f94ec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f94ec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717c28: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717c38: 0x7f9493f94ec0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717c20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 208 (Thread 0x7f9433a48640 (LWP 2739813) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f94e40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f94e40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717c08: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717c18: 0x7f9493f94e40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717c00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 207 (Thread 0x7f9434449640 (LWP 2739812) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f7a900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f7a900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717be8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717bf8: 0x7f9493f7a900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717be0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 206 (Thread 0x7f9434e4a640 (LWP 2739811) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f6dd40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f6dd40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717bc8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717bd8: 0x7f9493f6dd40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717bc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 205 (Thread 0x7f943584b640 (LWP 2739810) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f6dcc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f6dcc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717ba8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717bb8: 0x7f9493f6dcc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 204 (Thread 0x7f943624c640 (LWP 2739809) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f6da00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f6da00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717b88: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717b98: 0x7f9493f6da00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717b80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 203 (Thread 0x7f9436c4d640 (LWP 2739808) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f6d680) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f6d680) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717b68: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717b78: 0x7f9493f6d680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717b60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 202 (Thread 0x7f943764e640 (LWP 2739807) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f6d600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f6d600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717b48: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717b58: 0x7f9493f6d600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 201 (Thread 0x7f943804f640 (LWP 2739806) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f60f80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f60f80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717b28: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717b38: 0x7f9493f60f80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717b20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 200 (Thread 0x7f9438a50640 (LWP 2739805) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f606c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f606c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717b08: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717b18: 0x7f9493f606c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717b00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 199 (Thread 0x7f9439451640 (LWP 2739804) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f60640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f60640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717ae8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717af8: 0x7f9493f60640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717ae0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 198 (Thread 0x7f9439e52640 (LWP 2739803) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f3dd40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f3dd40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717ac8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717ad8: 0x7f9493f3dd40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717ac0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 197 (Thread 0x7f943a853640 (LWP 2739802) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f3d600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f3d600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717aa8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717ab8: 0x7f9493f3d600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717aa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 196 (Thread 0x7f943b254640 (LWP 2739801) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f3d580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f3d580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717a88: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717a98: 0x7f9493f3d580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 195 (Thread 0x7f943bc55640 (LWP 2739800) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f3d440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f3d440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717a68: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717a78: 0x7f9493f3d440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717a60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 194 (Thread 0x7f943c656640 (LWP 2739799) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f2d1c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f2d1c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717a48: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717a58: 0x7f9493f2d1c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717a40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 193 (Thread 0x7f943d057640 (LWP 2739798) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f2d140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f2d140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717a28: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717a38: 0x7f9493f2d140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717a20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 192 (Thread 0x7f943da58640 (LWP 2739797) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f20400) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f20400) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717a08: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717a18: 0x7f9493f20400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 191 (Thread 0x7f943e459640 (LWP 2739796) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06d00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06d00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937179e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937179f8: 0x7f9493f06d00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937179e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 190 (Thread 0x7f943ee5a640 (LWP 2739795) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06c80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06c80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937179c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937179d8: 0x7f9493f06c80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937179c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 189 (Thread 0x7f943f85b640 (LWP 2739794) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06a80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06a80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937179a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937179b8: 0x7f9493f06a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937179a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 188 (Thread 0x7f944025c640 (LWP 2739793) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f067c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f067c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717988: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717998: 0x7f9493f067c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717980) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 187 (Thread 0x7f9440c5d640 (LWP 2739792) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717968: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717978: 0x7f9493f06740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717960) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 186 (Thread 0x7f944165e640 (LWP 2739791) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717948: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717958: 0x7f9493f06540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 185 (Thread 0x7f944205f640 (LWP 2739790) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f06340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f06340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717928: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717938: 0x7f9493f06340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717920) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 184 (Thread 0x7f9442a60640 (LWP 2739789) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493f062c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493f062c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717908: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717918: 0x7f9493f062c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 183 (Thread 0x7f9443461640 (LWP 2739788) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ef9c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ef9c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937178e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937178f8: 0x7f9493ef9c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937178e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 182 (Thread 0x7f9443e62640 (LWP 2739787) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ef9800) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ef9800) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937178c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937178d8: 0x7f9493ef9800) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937178c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 181 (Thread 0x7f9444863640 (LWP 2739786) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ef9780) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ef9780) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937178a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937178b8: 0x7f9493ef9780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937178a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 180 (Thread 0x7f9445264640 (LWP 2739785) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ef9280) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ef9280) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717888: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717898: 0x7f9493ef9280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717880) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 179 (Thread 0x7f9445c65640 (LWP 2739784) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee9cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee9cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717868: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717878: 0x7f9493ee9cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717860) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 178 (Thread 0x7f9446666640 (LWP 2739783) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee9c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee9c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717848: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717858: 0x7f9493ee9c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 177 (Thread 0x7f9447067640 (LWP 2739782) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee98c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee98c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717828: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717838: 0x7f9493ee98c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717820) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 176 (Thread 0x7f9447a68640 (LWP 2739781) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee96c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee96c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717808: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717818: 0x7f9493ee96c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717800) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 175 (Thread 0x7f9448469640 (LWP 2739780) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee9640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee9640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937177e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937177f8: 0x7f9493ee9640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937177e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 174 (Thread 0x7f9448e6a640 (LWP 2739779) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ee9380) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ee9380) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937177c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937177d8: 0x7f9493ee9380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937177c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 173 (Thread 0x7f944986b640 (LWP 2739778) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed9a00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed9a00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937177a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937177b8: 0x7f9493ed9a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937177a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 172 (Thread 0x7f944a26c640 (LWP 2739777) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed9980) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed9980) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717788: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717798: 0x7f9493ed9980) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 171 (Thread 0x7f944ac6d640 (LWP 2739776) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed9840) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed9840) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717768: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717778: 0x7f9493ed9840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717760) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 170 (Thread 0x7f944b66e640 (LWP 2739775) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed97c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed97c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717748: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717758: 0x7f9493ed97c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 169 (Thread 0x7f944c06f640 (LWP 2739774) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed9740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed9740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717728: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717738: 0x7f9493ed9740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 168 (Thread 0x7f944ca70640 (LWP 2739773) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed93c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed93c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717708: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717718: 0x7f9493ed93c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717700) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 167 (Thread 0x7f944d471640 (LWP 2739772) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ed9040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ed9040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937176e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937176f8: 0x7f9493ed9040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937176e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 166 (Thread 0x7f944de72640 (LWP 2739771) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ecffc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ecffc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937176c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937176d8: 0x7f9493ecffc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937176c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 165 (Thread 0x7f944e873640 (LWP 2739770) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebfb00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebfb00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937176a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937176b8: 0x7f9493ebfb00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937176a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 164 (Thread 0x7f944f274640 (LWP 2739769) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebf9c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebf9c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717688: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717698: 0x7f9493ebf9c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 163 (Thread 0x7f944fc75640 (LWP 2739768) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebf940) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebf940) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717668: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717678: 0x7f9493ebf940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717660) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 162 (Thread 0x7f9450676640 (LWP 2739767) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebf740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebf740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717648: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717658: 0x7f9493ebf740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 161 (Thread 0x7f9451077640 (LWP 2739766) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebf600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebf600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717628: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717638: 0x7f9493ebf600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717620) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 160 (Thread 0x7f9451a78640 (LWP 2739765) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ebf580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ebf580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717608: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717618: 0x7f9493ebf580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 159 (Thread 0x7f9452479640 (LWP 2739764) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493eb2c00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493eb2c00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937175e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937175f8: 0x7f9493eb2c00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937175e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 158 (Thread 0x7f9452e7a640 (LWP 2739763) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493eb2340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493eb2340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937175c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937175d8: 0x7f9493eb2340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937175c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 157 (Thread 0x7f945387b640 (LWP 2739762) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493eb22c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493eb22c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937175a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937175b8: 0x7f9493eb22c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937175a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 156 (Thread 0x7f945427c640 (LWP 2739761) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2f40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2f40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717588: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717598: 0x7f9493ea2f40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 155 (Thread 0x7f9454c7d640 (LWP 2739760) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2b00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2b00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717568: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717578: 0x7f9493ea2b00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717560) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 154 (Thread 0x7f945567e640 (LWP 2739759) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2a80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2a80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717548: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717558: 0x7f9493ea2a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 153 (Thread 0x7f945607f640 (LWP 2739758) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717528: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717538: 0x7f9493ea2640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717520) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 152 (Thread 0x7f9456a80640 (LWP 2739757) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea20c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea20c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717508: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717518: 0x7f9493ea20c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 151 (Thread 0x7f9457481640 (LWP 2739756) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2400) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2400) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937174e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937174f8: 0x7f9493ea2400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937174e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 150 (Thread 0x7f9457e82640 (LWP 2739755) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea22c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea22c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937174c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937174d8: 0x7f9493ea22c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937174c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 149 (Thread 0x7f9458883640 (LWP 2739754) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2240) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2240) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937174a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937174b8: 0x7f9493ea2240) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937174a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 148 (Thread 0x7f9459284640 (LWP 2739753) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493ea2100) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493ea2100) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717448: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717458: 0x7f9493ea2100) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 147 (Thread 0x7f9459c85640 (LWP 2739752) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e95e00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e95e00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717488: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717498: 0x7f9493e95e00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 146 (Thread 0x7f945a686640 (LWP 2739751) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e95cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e95cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717468: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717478: 0x7f9493e95cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717460) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 145 (Thread 0x7f945b087640 (LWP 2739750) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e95c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e95c40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717428: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717438: 0x7f9493e95c40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717420) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 144 (Thread 0x7f945ba88640 (LWP 2739749) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e95200) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e95200) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493717408: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493717418: 0x7f9493e95200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493717400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 143 (Thread 0x7f945c489640 (LWP 2739748) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e280, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e950c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e950c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94937173e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94937173f8: 0x7f9493e950c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94937173e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 142 (Thread 0x7f945ce8a640 (LWP 2739747) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88e80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88e80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4ac8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4ad8: 0x7f9493e88e80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4ac0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 141 (Thread 0x7f945d88b640 (LWP 2739746) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88e00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88e00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4aa8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4ab8: 0x7f9493e88e00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4aa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 140 (Thread 0x7f945e28c640 (LWP 2739745) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88cc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4a88: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4a98: 0x7f9493e88cc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4a80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 139 (Thread 0x7f945ec8d640 (LWP 2739744) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e887c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e887c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4a68: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4a78: 0x7f9493e887c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4a60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 138 (Thread 0x7f945f68e640 (LWP 2739743) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4a48: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4a58: 0x7f9493e88740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4a40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 137 (Thread 0x7f946008f640 (LWP 2739742) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4a28: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4a38: 0x7f9493e88600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4a20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 136 (Thread 0x7f9460a90640 (LWP 2739741) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e881c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e881c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4a08: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4a18: 0x7f9493e881c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 135 (Thread 0x7f9461491640 (LWP 2739740) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb49e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb49f8: 0x7f9493e88140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb49e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 134 (Thread 0x7f9461e92640 (LWP 2739739) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e88000) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e88000) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb49c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb49d8: 0x7f9493e88000) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb49c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 133 (Thread 0x7f9462893640 (LWP 2739738) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6cec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6cec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb49a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb49b8: 0x7f9493e6cec0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb49a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 132 (Thread 0x7f9463294640 (LWP 2739737) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c9c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c9c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4988: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4998: 0x7f9493e6c9c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4980) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 131 (Thread 0x7f9463c95640 (LWP 2739736) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c940) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c940) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4968: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4978: 0x7f9493e6c940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4960) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 130 (Thread 0x7f9464696640 (LWP 2739735) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c8c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c8c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4948: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4958: 0x7f9493e6c8c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4940) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 129 (Thread 0x7f9465097640 (LWP 2739734) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c6c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c6c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4928: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4938: 0x7f9493e6c6c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4920) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 128 (Thread 0x7f9465a98640 (LWP 2739733) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c640) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4908: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4918: 0x7f9493e6c640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 127 (Thread 0x7f9466499640 (LWP 2739732) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb48e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb48f8: 0x7f9493e6c440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb48e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 126 (Thread 0x7f9466e9a640 (LWP 2739731) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c300) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c300) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb48c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb48d8: 0x7f9493e6c300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb48c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 125 (Thread 0x7f946789b640 (LWP 2739730) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c280) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c280) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb48a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb48b8: 0x7f9493e6c280) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb48a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 124 (Thread 0x7f946829c640 (LWP 2739729) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e6c080) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e6c080) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4888: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4898: 0x7f9493e6c080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4880) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 123 (Thread 0x7f9468c9d640 (LWP 2739728) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4868: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4878: 0x7f9493e14fc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4860) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 122 (Thread 0x7f946969e640 (LWP 2739727) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14d00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14d00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4848: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4858: 0x7f9493e14d00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 121 (Thread 0x7f946a09f640 (LWP 2739726) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14bc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14bc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4828: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4838: 0x7f9493e14bc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4820) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 120 (Thread 0x7f946aaa0640 (LWP 2739725) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14b40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14b40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4808: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4818: 0x7f9493e14b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4800) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 119 (Thread 0x7f946b4a1640 (LWP 2739724) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14a00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14a00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb47e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb47f8: 0x7f9493e14a00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb47e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 118 (Thread 0x7f946bea2640 (LWP 2739723) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e148c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e148c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb47c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb47d8: 0x7f9493e148c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb47c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 117 (Thread 0x7f946c8a3640 (LWP 2739722) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14840) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14840) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb47a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb47b8: 0x7f9493e14840) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb47a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 116 (Thread 0x7f946d2a4640 (LWP 2739721) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4788: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4798: 0x7f9493e14340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4780) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 115 (Thread 0x7f946dca5640 (LWP 2739720) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14200) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14200) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4768: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4778: 0x7f9493e14200) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4760) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 114 (Thread 0x7f946e6a6640 (LWP 2739719) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493e14180) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493e14180) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4748: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4758: 0x7f9493e14180) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 113 (Thread 0x7f946f0a7640 (LWP 2739718) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df8540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df8540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4728: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4738: 0x7f9493df8540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4720) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 112 (Thread 0x7f946faa8640 (LWP 2739717) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3fc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4708: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4718: 0x7f9493df3fc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4700) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 111 (Thread 0x7f94704a9640 (LWP 2739716) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3dc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3dc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb46e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb46f8: 0x7f9493df3dc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb46e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 110 (Thread 0x7f9470eaa640 (LWP 2739715) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3bc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3bc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb46c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb46d8: 0x7f9493df3bc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb46c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 109 (Thread 0x7f94718ab640 (LWP 2739714) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3b40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3b40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb46a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb46b8: 0x7f9493df3b40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb46a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 108 (Thread 0x7f94722ac640 (LWP 2739713) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df37c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df37c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4688: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4698: 0x7f9493df37c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 107 (Thread 0x7f9472cad640 (LWP 2739712) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3680) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3680) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4668: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4678: 0x7f9493df3680) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4660) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 106 (Thread 0x7f94736ae640 (LWP 2739711) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4648: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4658: 0x7f9493df3600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4640) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 105 (Thread 0x7f94740af640 (LWP 2739710) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df34c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df34c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4628: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4638: 0x7f9493df34c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4620) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 104 (Thread 0x7f9474ab0640 (LWP 2739709) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3440) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4608: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4618: 0x7f9493df3440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 103 (Thread 0x7f94754b1640 (LWP 2739708) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df33c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df33c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb45e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb45f8: 0x7f9493df33c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb45e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 102 (Thread 0x7f9475eb2640 (LWP 2739707) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3340) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb45c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb45d8: 0x7f9493df3340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb45c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 101 (Thread 0x7f94768b3640 (LWP 2739706) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493df3040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493df3040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb45a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb45b8: 0x7f9493df3040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb45a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 100 (Thread 0x7f94772b4640 (LWP 2739705) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493deeec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493deeec0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4588: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4598: 0x7f9493deeec0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 99 (Thread 0x7f9477cb5640 (LWP 2739704) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493deed00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493deed00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4568: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4578: 0x7f9493deed00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4560) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 98 (Thread 0x7f94786b6640 (LWP 2739703) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dee880) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dee880) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4548: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4558: 0x7f9493dee880) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 97 (Thread 0x7f94790b7640 (LWP 2739702) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dee6c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dee6c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4528: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4538: 0x7f9493dee6c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4520) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 96 (Thread 0x7f9479ab8640 (LWP 2739701) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dee300) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dee300) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4508: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4518: 0x7f9493dee300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4500) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 95 (Thread 0x7f947a4b9640 (LWP 2739700) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dead80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dead80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb44e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb44f8: 0x7f9493dead80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb44e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 94 (Thread 0x7f947aeba640 (LWP 2739699) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493deabc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493deabc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb44c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb44d8: 0x7f9493deabc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb44c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 93 (Thread 0x7f947b8bb640 (LWP 2739698) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dea5c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dea5c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb44a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb44b8: 0x7f9493dea5c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb44a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 92 (Thread 0x7f947c2bc640 (LWP 2739697) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dea2c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dea2c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4488: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4498: 0x7f9493dea2c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 91 (Thread 0x7f947ccbd640 (LWP 2739696) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493dea140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493dea140) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4468: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4478: 0x7f9493dea140) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4460) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 90 (Thread 0x7f947d6be640 (LWP 2739695) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493de2b00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493de2b00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4448: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4458: 0x7f9493de2b00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4440) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 89 (Thread 0x7f947e0bf640 (LWP 2739694) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493de2540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493de2540) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4428: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4438: 0x7f9493de2540) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4420) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 88 (Thread 0x7f947eac0640 (LWP 2739693) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f9493de2480) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f9493de2480) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4408: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4418: 0x7f9493de2480) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4400) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 87 (Thread 0x7f947f4c1640 (LWP 2739692) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938edb80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938edb80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb43e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb43f8: 0x7f94938edb80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb43e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 86 (Thread 0x7f947fec2640 (LWP 2739691) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938edb00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938edb00) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb43c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb43d8: 0x7f94938edb00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb43c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 85 (Thread 0x7f94808c3640 (LWP 2739690) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb43a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb43b8: 0x7f94938ed900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb43a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 84 (Thread 0x7f94812c4640 (LWP 2739689) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed7c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed7c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4388: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4398: 0x7f94938ed7c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4380) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 83 (Thread 0x7f9481cc5640 (LWP 2739688) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed740) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4368: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4378: 0x7f94938ed740) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4360) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 82 (Thread 0x7f94826c6640 (LWP 2739687) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed3c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed3c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4348: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4358: 0x7f94938ed3c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4340) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 81 (Thread 0x7f94830c7640 (LWP 2739686) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed1c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed1c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4328: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4338: 0x7f94938ed1c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4320) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 80 (Thread 0x7f9483ac8640 (LWP 2739685) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebff600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebff600) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4308: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4318: 0x7f956ebff600) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4300) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 79 (Thread 0x7f94844c9640 (LWP 2739684) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f94938ed040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f94938ed040) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb42e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb42f8: 0x7f94938ed040) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb42e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 78 (Thread 0x7f9484eca640 (LWP 2739683) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebfff40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebfff40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb42c8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb42d8: 0x7f956ebfff40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb42c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 77 (Thread 0x7f94858cb640 (LWP 2739682) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebffbc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebffbc0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb42a8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb42b8: 0x7f956ebffbc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb42a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 76 (Thread 0x7f94862cc640 (LWP 2739681) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebffa40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebffa40) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94bd819068: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94bd819078: 0x7f956ebffa40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94bd819060) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 75 (Thread 0x7f9486ccd640 (LWP 2739680) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebff580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebff580) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f956eb788e8: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f956eb788f8: 0x7f956ebff580) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f956eb788e0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 74 (Thread 0x7f94876ce640 (LWP 2739679) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebfed80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebfed80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f956ebeac48: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f956ebeac58: 0x7f956ebfed80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f956ebeac40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 73 (Thread 0x7f94880cf640 (LWP 2739678) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebfea80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebfea80) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94bd819088: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94bd819098: 0x7f956ebfea80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94bd819080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 72 (Thread 0x7f9488ad0640 (LWP 2739677) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebfe900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebfe900) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f9493fb4228: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f9493fb4238: 0x7f956ebfe900) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f9493fb4220) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 71 (Thread 0x7f94894d1640 (LWP 2739676) "SegmentReader"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c2333852f in DB::DM::WorkQueue >::pop (this=0x7f957021e0c0, item=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h:140 -#4 0x0000558c23337179 in DB::DM::SegmentReader::readSegments (this=0x7f956ebfe5c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:58 -#5 0x0000558c23336f99 in DB::DM::SegmentReader::run (this=0x7f956ebfmpptaske5c0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp:122 -#6 0x0000558c2333a219 in std::__1::__invoke[abi:ue170006](void (DB::DM::SegmentReader::*&&)(), DB::DM::SegmentReader*&&) (__f=@0x7f94bd819008: (void (DB::DM::SegmentReader::*)(class DB::DM::SegmentReader * const)) 0x558c23336f40 , __a0=@0x7f94bd819018: 0x7f956ebfe5c0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#7 0x0000558c2333a19e in std::__1::__thread_execute[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*, 2ul>(std::__1::tuple >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c23339f72 in std::__1::__thread_proxy[abi:ue170006] >, void (DB::DM::SegmentReader::*)(), DB::DM::SegmentReader*> >(void*) (__vp=0x7f94bd819000) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 70 (Thread 0x7f9489ed2640 (LWP 2739675) "MPPTask-Moniter"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f9493f9c900, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abcc52f in std::__1::condition_variable::wait_for > (this=0x7f9493f9c900, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c26907e86 in DB::(anonymous namespace)::monitorMPPTasks (monitor=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/TMTContext.cpp:103 -#5 0x0000558c26907dd6 in DB::(anonymous namespace)::startMonitorMPPTaskThread(std::__1::shared_ptr const&)::$_0::operator()() const (this=0x7f956f9af1a8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/TMTContext.cpp:123 -#6 0x0000558c26907da5 in std::__1::__invoke[abi:ue170006] const&)::$_0&>(DB::(anonymous namespace)::startMonitorMPPTaskThread(std::__1::shared_ptr const&)::$_0&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c26907d65 in std::__1::__invoke_void_return_wrapper::__call[abi:ue170006] const&)::$_0&>(DB::(anonymous namespace)::startMonitorMPPTaskThread(std::__1::shared_ptr const&)::$_0&) (__args=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:415 -#8 0x0000558c26907d3d in std::__1::__function::__alloc_func const&)::$_0, std::__1::allocator const&)::$_0>, void ()>::operator()[abi:ue170006]() (this=0x7f956f9af1a8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:192 -#9 0x0000558c26907039 in std::__1::__function::__func const&)::$_0, std::__1::allocator const&)::$_0>, void ()>::operator()() (this=0x7f956f9af1a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:363 -#10 0x0000558c1abb48d2 in std::__1::__function::__value_func::operator()[abi:ue170006]() const (this=0x7f956f9af1a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:517 -#11 0x0000558c1aba7925 in std::__1::function::operator()() const (this=0x7f956f9af1a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:1168 -#12 0x0000558c238349d5 in std::__1::__invoke[abi:ue170006] const&>(std::__1::function const&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#13 0x0000558c23834975 in std::__1::invoke[abi:ue170006] const&>(std::__1::function const&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/invoke.h:30 -#14 0x0000558c238348f3 in DB::ThreadFactory::newThread>(bool, std::__1::basic_string, std::__1::allocator >, std::__1::function&&)::{lambda((auto:1&&)...)#1}::operator()<>() const (this=0x7f956f9af170) at /DATA/disk3/xzx/tiflash/dbms/src/Common/ThreadFactory.h:48 -#15 0x0000558c23834775 in std::__1::__invoke[abi:ue170006]>(bool, std::__1::basic_string, std::__1::allocator >, std::__1::function&&)::{lambda((auto:1&&)...)#1}>(std::__1::function&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#16 0x0000558c2383474d in std::__1::__thread_execute[abi:ue170006] >, DB::ThreadFactory::newThread>(bool, std::__1::basic_string, std::__1::allocator >, std::__1::function&&)::{lambda((auto:1&&)...)#1}>(std::__1::tuple >, DB::ThreadFactory::newThread>(bool, std::__1::basic_string, std::__1::allocator >, std::__1::function&&)::{lambda((auto:1&&)...)#1}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#17 0x0000558c238344d2 in std::__1::__thread_proxy[abi:ue170006] >, DB::ThreadFactory::newThread>(bool, std::__1::basic_string, std::__1::allocator >, std::__1::function&&)::{lambda((auto:1&&)...)#1}> >(void*) (__vp=0x7f956f9af160) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 69 (Thread 0x7f948a8d3640 (LWP 2739674) "TiFlashMain"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f872688, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f956f872688, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c27b5afe9 in std::__1::condition_variable::wait_until >, pingcap::common::MPPProber::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, pingcap::common::MPPProber::run()::$_0) (this=0x7f956f872688, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c27b5a59f in std::__1::condition_variable::wait_for[abi:ue170006], pingcap::common::MPPProber::run()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, pingcap::common::MPPProber::run()::$_0) (this=0x7f956f872688, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 -#6 0x0000558c27b5a4f5 in pingcap::common::MPPProber::run (this=0x7f956f8725e0) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/common/MPPProber.cc:53 -#7 0x0000558c27ae93f4 in pingcap::kv::Cluster::startBackgroundTasks()::$_0::operator()() const (this=0x7f948a89fba8) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/kv/Cluster.cc:34 -#8 0x0000558c27ae93c5 in std::__1::__invoke[abi:ue170006](pingcap::kv::Cluster::startBackgroundTasks()::$_0&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#9 0x0000558c27ae9385 in std::__1::__invoke_void_return_wrapper::__call[abi:ue170006](pingcap::kv::Cluster::startBackgroundTasks()::$_0&) (__args=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:415 -#10 0x0000558c27ae935d in std::__1::__function::__alloc_func, void ()>::operator()[abi:ue170006]() (this=0x7f948a89fba8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:192 -#11 0x0000558c27ae8769 in std::__1::__function::__func, void ()>::operator()() (this=0x7f948a89fba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:363 -#12 0x0000558c1abb48d2 in std::__1::__function::__value_func::operator()[abi:ue170006]() const (this=0x7f948a89fba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:517 -#13 0x0000558c1aba7925 in std::__1::function::operator()() const (this=0x7f948a89fba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:1168 -#14 0x0000558c27b593af in pingcap::common::FixedThreadPool::loop (this=0x7f9493f9ddc0) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/common/FixedThreadPool.cc:33 -#15 0x0000558c27b59ae9 in std::__1::__invoke[abi:ue170006](void (pingcap::common::FixedThreadPool::*&&)(), pingcap::common::FixedThreadPool*&&) (__f=@0x7f956ebf3008: (void (pingcap::common::FixedThreadPool::*)(class pingcap::common::FixedThreadPool * const)) 0x558c27b592b0 , __a0=@0x7f956ebf3018: 0x7f9493f9ddc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#16 0x0000558c27b59a6e in std::__1::__thread_execute[abi:ue170006] >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*, 2ul>(std::__1::tuple >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#17 0x0000558c27b59842 in std::__1::__thread_proxy[abi:ue170006] >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*> >(void*) (__vp=0x7f956ebf3000) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 68 (Thread 0x7f948b2d4640 (LWP 2739673) "TiFlashMain"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f8c33b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f956f8c33b0, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c27aec349 in std::__1::condition_variable::wait_until >, pingcap::kv::RegionCache::updateCachePeriodically()::{lambda()#1}>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, pingcap::kv::RegionCache::updateCachePeriodically()::{lambda()#1}) (this=0x7f956f8c33b0, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c27aec2ef in std::__1::condition_variable::wait_for[abi:ue170006], pingcap::kv::RegionCache::updateCachePeriodically()::{lambda()#1}>(std::__1::unique_lock&, std::__1::chrono::duration > const&, pingcap::kv::RegionCache::updateCachePeriodically()::{lambda()#1}) (this=0x7f956f8c33b0, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 -#6 0x0000558c27aec24d in pingcap::kv::RegionCache::updateCachePeriodically (this=0x7f956f8c3200) at /DATA/disk3/xzx/tiflash/contrib/client-c/include/pingcap/kv/RegionCache.h:244 -#7 0x0000558c27aea4f4 in pingcap::kv::Cluster::startBackgroundTasks()::$_1::operator()() const (this=0x7f948b2a0ba8) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/kv/Cluster.cc:41 -#8 0x0000558c27aea4c5 in std::__1::__invoke[abi:ue170006](pingcap::kv::Cluster::startBackgroundTasks()::$_1&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#9 0x0000558c27aea485 in std::__1::__invoke_void_return_wrapper::__call[abi:ue170006](pingcap::kv::Cluster::startBackgroundTasks()::$_1&) (__args=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:415 -#10 0x0000558c27aea45d in std::__1::__function::__alloc_func, void ()>::operator()[abi:ue170006]() (this=0x7f948b2a0ba8) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:192 -#11 0x0000558c27ae9869 in std::__1::__function::__func, void ()>::operator()() (this=0x7f948b2a0ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:363 -#12 0x0000558c1abb48d2 in std::__1::__function::__value_func::operator()[abi:ue170006]() const (this=0x7f948b2a0ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:517 -#13 0x0000558c1aba7925 in std::__1::function::operator()() const (this=0x7f948b2a0ba0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__functional/function.h:1168 -#14 0x0000558c27b593af in pingcap::common::FixedThreadPool::loop (this=0x7f9493f9ddc0) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/common/FixedThreadPool.cc:33 -#15 0x0000558c27b59ae9 in std::__1::__invoke[abi:ue170006](void (pingcap::common::FixedThreadPool::*&&)(), pingcap::common::FixedThreadPool*&&) (__f=@0x7f956ebeacc8: (void (pingcap::common::FixedThreadPool::*)(class pingcap::common::FixedThreadPool * const)) 0x558c27b592b0 , __a0=@0x7f956ebeacd8: 0x7f9493f9ddc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:308 -#16 0x0000558c27b59a6e in std::__1::__thread_execute[abi:ue170006] >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*, 2ul>(std::__1::tuple >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#17 0x0000558c27b59842 in std::__1::__thread_proxy[abi:ue170006] >, void (pingcap::common::FixedThreadPool::*)(), pingcap::common::FixedThreadPool*> >(void*) (__vp=0x7f956ebeacc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 67 (Thread 0x7f948bcd5640 (LWP 2739672) "PDUpdateTS"): -#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 -#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 -#2 0x00007f95847adc8b in std::__1::this_thread::sleep_for(std::__1::chrono::duration > const&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ab783b2 in std::__1::this_thread::sleep_for[abi:ue170006] >(std::__1::chrono::duration > const&) (__d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/this_thread.h:55 -#4 0x0000558c2691b43d in pingcap::pd::Oracle::updateTS (this=0x7f956ebf79c0, update_interval=...) at /DATA/disk3/xzx/tiflash/contrib/client-c/include/pingcap/pd/Oracle.h:79 -#5 0x0000558c2691b324 in pingcap::pd::Oracle::Oracle(std::__1::shared_ptr, std::__1::chrono::duration >)::{lambda()#1}::operator()() const (this=0x7f9493def088) at /DATA/disk3/xzx/tiflash/contrib/client-c/include/pingcap/pd/Oracle.h:42 -#6 0x0000558c2691b2d5 in std::__1::__invoke[abi:ue170006], std::__1::chrono::duration >)::{lambda()#1}>(pingcap::pd::Oracle::Oracle(std::__1::shared_ptr, std::__1::chrono::duration >)::{lambda()#1}&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c2691b2ad in std::__1::__thread_execute[abi:ue170006] >, pingcap::pd::Oracle::Oracle(std::__1::shared_ptr, std::__1::chrono::duration >)::{lambda()#1}>(std::__1::tuple >, pingcap::pd::Oracle::Oracle(std::__1::shared_ptr, std::__1::chrono::duration >)::{lambda()#1}>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c2691b0e2 in std::__1::__thread_proxy[abi:ue170006] >, pingcap::pd::Oracle::Oracle(std::__1::shared_ptr, std::__1::chrono::duration >)::{lambda()#1}> >(void*) (__vp=0x7f9493def080) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 66 (Thread 0x7f948c6d6640 (LWP 2739671) "PDLeaderLoop"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f957068996f in pthread_cond_timedwait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799cda in std::__1::condition_variable::__do_timed_wait(std::__1::unique_lock&, std::__1::chrono::time_point > >) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c27b346bb in std::__1::condition_variable::wait_until > > (this=0x7f95702af738, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c27b26ea9 in std::__1::condition_variable::wait_until >, pingcap::pd::Client::leaderLoop()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, pingcap::pd::Client::leaderLoop()::$_0) (this=0x7f95702af738, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c27b26c31 in pingcap::pd::Client::leaderLoop (this=0x7f95702af598) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/pd/Client.cc:254 -#6 0x0000558c27b2bce8 in pingcap::pd::Client::Client(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0::operator()() const (this=0x7f94bd815018) at /DATA/disk3/xzx/tiflash/contrib/client-c/src/pd/Client.cc:56 -#7 0x0000558c27b2bca5 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0>(pingcap::pd::Client::Client(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c27b2bc7d in std::__1::__thread_execute[abi:ue170006] >, pingcap::pd::Client::Client(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0>(std::__1::tuple >, pingcap::pd::Client::Client(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c27b2bab2 in std::__1::__thread_proxy[abi:ue170006] >, pingcap::pd::Client::Client(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, pingcap::ClusterConfig const&)::$_0> >(void*) (__vp=0x7f94bd815010) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 65 (Thread 0x7f948d0d7640 (LWP 2739670) "grpc_global_tim"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x0000558c288f9efe in absl::lts_20211102::synchronization_internal::FutexImpl::WaitUntil (v=0x7f958456cf40, val=0, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/futex.h:104 -#2 0x0000558c288f9bfb in absl::lts_20211102::synchronization_internal::Waiter::Wait (this=0x7f958456cf40, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/waiter.cc:95 -#3 0x0000558c288f9816 in AbslInternalPerThreadSemWait_lts_20211102 (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.cc:93 -#4 0x0000558c288f6bbd in absl::lts_20211102::synchronization_internal::PerThreadSem::Wait (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.h:112 -#5 0x0000558c288efd56 in absl::lts_20211102::Mutex::DecrementSynchSem (mu=0x558c29567628 , w=0x7f958456cf00, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:579 -#6 0x0000558c288f4e7a in absl::lts_20211102::CondVar::WaitCommon (this=0x558c29567630 , mutex=0x558c29567628 , t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2559 -#7 0x0000558c288f50aa in absl::lts_20211102::CondVar::Wait (this=0x558c29567630 , mu=0x558c29567628 ) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2594 -#8 0x0000558c2832ff8b in gpr_cv_wait (cv=0x558c29567630 , mu=0x558c29567628 , abs_deadline=...) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gpr/sync_abseil.cc:85 -#9 0x0000558c27e19621 in wait_until (next=9223372036854775807) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:200 -#10 0x0000558c27e191e9 in timer_main_loop () at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:255 -#11 0x0000558c27e190ef in timer_thread (completed_thread_ptr=0x7f9493e250e0) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/timer_manager.cc:284 -#12 0x0000558c28339f22 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::operator()(void*) const (this=0x7f948d0a3c87, v=0x7f9493e25140) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:140 -#13 0x0000558c28339e09 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) (v=0x7f9493e25140) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:110 -#14 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#15 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 64 (Thread 0x7f948dad8640 (LWP 2739669) "resolver-execut"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x0000558c288f9efe in absl::lts_20211102::synchronization_internal::FutexImpl::WaitUntil (v=0x7f958456c940, val=0, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/futex.h:104 -#2 0x0000558c288f9bfb in absl::lts_20211102::synchronization_internal::Waiter::Wait (this=0x7f958456c940, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/waiter.cc:95 -#3 0x0000558c288f9816 in AbslInternalPerThreadSemWait_lts_20211102 (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.cc:93 -#4 0x0000558c288f6bbd in absl::lts_20211102::synchronization_internal::PerThreadSem::Wait (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.h:112 -#5 0x0000558c288efd56 in absl::lts_20211102::Mutex::DecrementSynchSem (mu=0x7f9493fa6800, w=0x7f958456c900, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:579 -#6 0x0000558c288f4e7a in absl::lts_20211102::CondVar::WaitCommon (this=0x7f9493fa6818, mutex=0x7f9493fa6800, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2559 -#7 0x0000558c288f50aa in absl::lts_20211102::CondVar::Wait (this=0x7f9493fa6818, mu=0x7f9493fa6800) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2594 -#8 0x0000558c2832ff8b in gpr_cv_wait (cv=0x7f9493fa6818, mu=0x7f9493fa6800, abs_deadline=...) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gpr/sync_abseil.cc:85 -#9 0x0000558c27df194f in grpc_core::Executor::ThreadMain (arg=0x7f9493fa6800) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/executor.cc:238 -#10 0x0000558c28339f22 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::operator()(void*) const (this=0x7f948daa4c87, v=0x7f9493e25080) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:140 -#11 0x0000558c28339e09 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) (v=0x7f9493e25080) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:110 -#12 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#13 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 63 (Thread 0x7f948e4d9640 (LWP 2739668) "default-executo"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x0000558c288f9efe in absl::lts_20211102::synchronization_internal::FutexImpl::WaitUntil (v=0x7f958456c340, val=0, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/futex.h:104 -#2 0x0000558c288f9bfb in absl::lts_20211102::synchronization_internal::Waiter::Wait (this=0x7f958456c340, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/waiter.cc:95 -#3 0x0000558c288f9816 in AbslInternalPerThreadSemWait_lts_20211102 (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.cc:93 -#4 0x0000558c288f6bbd in absl::lts_20211102::synchronization_internal::PerThreadSem::Wait (t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/internal/per_thread_sem.h:112 -#5 0x0000558c288efd56 in absl::lts_20211102::Mutex::DecrementSynchSem (mu=0x7f9493fa3000, w=0x7f958456c300, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:579 -#6 0x0000558c288f4e7a in absl::lts_20211102::CondVar::WaitCommon (this=0x7f9493fa3018, mutex=0x7f9493fa3000, t=...) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2559 -#7 0x0000558c288f50aa in absl::lts_20211102::CondVar::Wait (this=0x7f9493fa3018, mu=0x7f9493fa3000) at /DATA/disk3/xzx/tiflash/contrib/abseil-cpp/absl/synchronization/mutex.cc:2594 -#8 0x0000558c2832ff8b in gpr_cv_wait (cv=0x7f9493fa3018, mu=0x7f9493fa3000, abs_deadline=...) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gpr/sync_abseil.cc:85 -#9 0x0000558c27df194f in grpc_core::Executor::ThreadMain (arg=0x7f9493fa3000) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/iomgr/executor.cc:238 -#10 0x0000558c28339f22 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::operator()(void*) const (this=0x7f948e4a5c87, v=0x7f9493e25020) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:140 -#11 0x0000558c28339e09 in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) (v=0x7f9493e25020) at /DATA/disk3/xzx/tiflash/contrib/grpc/src/core/lib/gprpp/thd_posix.cc:110 -#12 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#13 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 62 (Thread 0x7f948eeda640 (LWP 2739667) "TiFlashMain"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f95702af4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f95702af4b0, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c238923c9 in std::__1::condition_variable::wait_until >, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0) (this=0x7f95702af4b0, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c23891fff in std::__1::condition_variable::wait_for[abi:ue170006], DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0) (this=0x7f95702af4b0, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 -#6 0x0000558c23891ea0 in DB::DM::LocalIndexCache::cleanOutdatedLoop (this=0x7f95702af318) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/Index/LocalIndexCache.cpp:69 -#7 0x0000558c23892738 in DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0::operator()() const (this=0x7f9493def038) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/Index/LocalIndexCache.cpp:90 -#8 0x0000558c238926f5 in std::__1::__invoke[abi:ue170006](DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#9 0x0000558c238926cd in std::__1::__thread_execute[abi:ue170006] >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0>(std::__1::tuple >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#10 0x0000558c23892502 in std::__1::__thread_proxy[abi:ue170006] >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0> >(void*) (__vp=0x7f9493def030) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#11 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#12 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 61 (Thread 0x7f948f8db640 (LWP 2739666) "TiFlashMain"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f95702af230, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f95702af230, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c238923c9 in std::__1::condition_variable::wait_until >, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0) (this=0x7f95702af230, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c23891fff in std::__1::condition_variable::wait_for[abi:ue170006], DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::DM::LocalIndexCache::cleanOutdatedLoop()::$_0) (this=0x7f95702af230, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 -#6 0x0000558c23891ea0 in DB::DM::LocalIndexCache::cleanOutdatedLoop (this=0x7f95702af098) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/Index/LocalIndexCache.cpp:69 -#7 0x0000558c23892738 in DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0::operator()() const (this=0x7f9493def028) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/Index/LocalIndexCache.cpp:90 -#8 0x0000558c238926f5 in std::__1::__invoke[abi:ue170006](DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#9 0x0000558c238926cd in std::__1::__thread_execute[abi:ue170006] >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0>(std::__1::tuple >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#10 0x0000558c23892502 in std::__1::__thread_proxy[abi:ue170006] >, DB::DM::LocalIndexCache::LocalIndexCache(unsigned long)::$_0> >(void*) (__vp=0x7f9493def020) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#11 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#12 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 60 (Thread 0x7f94902dc640 (LWP 2739665) "TiFlashMain"): -#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 -#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 -#2 0x00007f95847adc8b in std::__1::this_thread::sleep_for(std::__1::chrono::duration > const&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c1ab783b2 in std::__1::this_thread::sleep_for[abi:ue170006] >(std::__1::chrono::duration > const&) (__d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/this_thread.h:55 -#4 0x0000558c1ac1aafa in DB::IORateLimiter::runAutoTune()::$_0::operator()() const (this=0x7f9493def018) at /DATA/disk3/xzx/tiflash/dbms/src/IO/BaseFile/RateLimiter.cpp:693 -#5 0x0000558c1ac1aa65 in std::__1::__invoke[abi:ue170006](DB::IORateLimiter::runAutoTune()::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#6 0x0000558c1ac1aa3d in std::__1::__thread_execute[abi:ue170006] >, DB::IORateLimiter::runAutoTune()::$_0>(std::__1::tuple >, DB::IORateLimiter::runAutoTune()::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#7 0x0000558c1ac1a872 in std::__1::__thread_proxy[abi:ue170006] >, DB::IORateLimiter::runAutoTune()::$_0> >(void*) (__vp=0x7f9493def010) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#8 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#9 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 59 (Thread 0x7f9492cdd640 (LWP 2739664) "LocalIndexSched"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689782 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x00007f9584799c4f in std::__1::condition_variable::wait(std::__1::unique_lock&) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c22d9cd31 in std::__1::condition_variable::wait(std::__1::unique_lock&, DB::DM::LocalIndexerScheduler::schedulerLoop()::$_0) (this=0x7f956f065c58, __lk=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:148 -#4 0x0000558c22d9cb46 in DB::DM::LocalIndexerScheduler::schedulerLoop (this=0x7f956f065b98) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.cpp:412 -#5 0x0000558c22d9d048 in DB::DM::LocalIndexerScheduler::start()::$_0::operator()() const (this=0x7f9570296fb8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.cpp:102 -#6 0x0000558c22d9d005 in std::__1::__invoke[abi:ue170006](DB::DM::LocalIndexerScheduler::start()::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#7 0x0000558c22d9cfdd in std::__1::__thread_execute[abi:ue170006] >, DB::DM::LocalIndexerScheduler::start()::$_0>(std::__1::tuple >, DB::DM::LocalIndexerScheduler::start()::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#8 0x0000558c22d9ce12 in std::__1::__thread_proxy[abi:ue170006] >, DB::DM::LocalIndexerScheduler::start()::$_0> >(void*) (__vp=0x7f9570296fb0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 58 (Thread 0x7f94949de640 (LWP 2739663) "bg-block-15"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=15) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bfc8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bfc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 57 (Thread 0x7f94953df640 (LWP 2739662) "bg-block-14"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=14) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bfa8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bfa0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 56 (Thread 0x7f9495de0640 (LWP 2739661) "bg-block-13"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=13) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bf88) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bf80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 55 (Thread 0x7f94967e1640 (LWP 2739660) "bg-block-12"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=12) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bf68) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bf60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 54 (Thread 0x7f94971e2640 (LWP 2739659) "bg-block-11"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=11) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bf48) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bf40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 53 (Thread 0x7f9497be3640 (LWP 2739658) "bg-block-10"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=10) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bf28) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bf20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 52 (Thread 0x7f94985e4640 (LWP 2739657) "bg-block-9"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=9) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bf08) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bf00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 51 (Thread 0x7f9498fe5640 (LWP 2739656) "bg-block-8"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bee8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bee0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 50 (Thread 0x7f94999e6640 (LWP 2739655) "bg-block-7"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=7) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bec8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bec0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 49 (Thread 0x7f949a3e7640 (LWP 2739654) "bg-block-6"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=6) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bea8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bea0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 48 (Thread 0x7f949ade8640 (LWP 2739653) "bg-block-5"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=5) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6be88) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6be80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 47 (Thread 0x7f949b7e9640 (LWP 2739652) "bg-block-4"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=4) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6be68) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6be60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 46 (Thread 0x7f949c1ea640 (LWP 2739651) "bg-block-3"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=3) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6be48) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6be40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 45 (Thread 0x7f949cbeb640 (LWP 2739650) "bg-block-2"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=2) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6be28) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6be20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 44 (Thread 0x7f949d5ec640 (LWP 2739649) "bg-block-1"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=1) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6be08) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6be00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 43 (Thread 0x7f949dfed640 (LWP 2739648) "bg-block-0"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d5f0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d5f0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d518, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d518, thread_idx=0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bde8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bde0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 42 (Thread 0x7f949e9ee640 (LWP 2739647) "bg-15"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=15) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bdc8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bdc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 41 (Thread 0x7f949f3ef640 (LWP 2739646) "bg-14"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=14) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bda8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bda0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 40 (Thread 0x7f949fdf0640 (LWP 2739645) "bg-13"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=13) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bd88) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bd80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 39 (Thread 0x7f94a07f1640 (LWP 2739644) "bg-12"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=12) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bd68) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bd60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 38 (Thread 0x7f94a11f2640 (LWP 2739643) "bg-11"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=11) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bd48) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bd40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 37 (Thread 0x7f94a1bf3640 (LWP 2739642) "bg-10"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=10) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bd28) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bd20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 36 (Thread 0x7f94a25f4640 (LWP 2739641) "bg-9"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=9) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bd08) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bd00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 35 (Thread 0x7f94a2ff5640 (LWP 2739640) "bg-8"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bce8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bce0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 34 (Thread 0x7f94a39f6640 (LWP 2739639) "bg-7"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=7) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bcc8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bcc0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 33 (Thread 0x7f94a43f7640 (LWP 2739638) "bg-6"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=6) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bca8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bca0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 32 (Thread 0x7f94a4df8640 (LWP 2739637) "bg-5"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=5) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bc88) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bc80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 31 (Thread 0x7f94a57f9640 (LWP 2739636) "bg-4"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=4) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bc68) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bc60) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 30 (Thread 0x7f94a61fa640 (LWP 2739635) "bg-3"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=3) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bc48) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bc40) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 29 (Thread 0x7f94a6bfb640 (LWP 2739634) "bg-2"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=2) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bc28) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bc20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 28 (Thread 0x7f94a75fc640 (LWP 2739633) "bg-1"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=1) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6bc08) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6bc00) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 27 (Thread 0x7f94a7ffd640 (LWP 2739632) "bg-0"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f956f09d4b0, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c24aedd1f in std::__1::condition_variable::wait_for > (this=0x7f956f09d4b0, __lk=..., __d=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:197 -#4 0x0000558c24ae98fb in DB::BackgroundProcessingPool::tryPopTask(pcg_detail::engine, false, pcg_detail::specific_stream, pcg_detail::default_multiplier >&) (this=0x7f956f09d3d8, rng=...) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:200 -#5 0x0000558c24ae9ca5 in DB::BackgroundProcessingPool::threadFunction (this=0x7f956f09d3d8, thread_idx=0) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:249 -#6 0x0000558c24aeac6c in DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1::operator()() const (this=0x7f956eb6b9a8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/BackgroundProcessingPool.cpp:101 -#7 0x0000558c24aeac25 in std::__1::__invoke[abi:ue170006], std::__1::allocator >, std::__1::shared_ptr)::$_1>(DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c24aeabfd in std::__1::__thread_execute[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>(std::__1::tuple >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c24aeaa22 in std::__1::__thread_proxy[abi:ue170006] >, DB::BackgroundProcessingPool::BackgroundProcessingPool(int, std::__1::basic_string, std::__1::allocator >, std::__1::shared_ptr)::$_1> >(void*) (__vp=0x7f956eb6b9a0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 26 (Thread 0x7f94a89fe640 (LWP 2739631) "ThdMemTrace"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f9570689b60 in pthread_cond_clockwait@GLIBC_2.30 () from /lib64/libc.so.6 -#2 0x0000558c1abae1d2 in std::__1::condition_variable::__do_timed_wait[abi:ue170006](std::__1::unique_lock&, std::__1::chrono::time_point > >) (this=0x7f957025c628, __lk=..., __tp=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:227 -#3 0x0000558c1abade37 in std::__1::condition_variable::wait_until > > (this=0x7f957025c628, __lk=..., __t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:162 -#4 0x0000558c2696ead9 in std::__1::condition_variable::wait_until >, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0::operator()() const::{lambda()#1}>(std::__1::unique_lock&, std::__1::chrono::time_point > > const&, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0::operator()() const::{lambda()#1}) (this=0x7f957025c628, __lk=..., __t=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:170 -#5 0x0000558c2696ea7f in std::__1::condition_variable::wait_for[abi:ue170006], DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0::operator()() const::{lambda()#1}>(std::__1::unique_lock&, std::__1::chrono::duration > const&, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0::operator()() const::{lambda()#1}) (this=0x7f957025c628, __lk=..., __d=..., __pred=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__condition_variable/condition_variable.h:206 -#6 0x0000558c2696e99d in DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0::operator()() const (this=0x7f957020efd8) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/FFI/JointThreadAllocInfo.cpp:39 -#7 0x0000558c2696e8f5 in std::__1::__invoke[abi:ue170006](DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0&&) (__f=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c2696e8cd in std::__1::__thread_execute[abi:ue170006] >, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0>(std::__1::tuple >, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0>&, std::__1::__tuple_indices<>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c2696e702 in std::__1::__thread_proxy[abi:ue170006] >, DB::JointThreadInfoJeallocMap::JointThreadInfoJeallocMap()::$_0> >(void*) (__vp=0x7f957020efd0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 25 (Thread 0x7f94ab1ff640 (LWP 2739630) "grpc_global_tim"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f958386d39d in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f95838674f0 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f9583917710 in timer_thread(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 24 (Thread 0x7f94d05ff640 (LWP 2739555) "background-3"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94d05fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94d05fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94d05fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689431152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94d05cac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94d05cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94d05cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 23 (Thread 0x7f94d0dff640 (LWP 2739554) "background-2"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94d0dfb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94d0dfb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94d0dfb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689431152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94d0dcac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94d0dcab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94d0dcb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 22 (Thread 0x7f94d13fe640 (LWP 2739553) "background-1"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94d13fa750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94d13fa750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94d13fa730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689431152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94d13c9c70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94d13c9b70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94d13ca170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 21 (Thread 0x7f94d15ff640 (LWP 2739552) "background-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f94d15fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f94d15fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f94d15fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279689431152, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f94d15cac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f94d15cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f94d15cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 20 (Thread 0x7f95625ff640 (LWP 2739550) "timer"): -#0 0x00007f9580aaadda in futures_channel::mpsc::UnboundedSenderInner>::inc_num_messages> (self=0x7f95625caff0) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-channel-0.3.31/src/mpsc/mod.rs:448 -#1 0x00007f9580aa9f11 in futures_channel::mpsc::UnboundedSender>::do_send_nb> (self=0x7f95625caff0, msg=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-channel-0.3.31/src/mpsc/mod.rs:793 -#2 0x00007f9580aaa0a7 in futures_channel::mpsc::UnboundedSender>::unbounded_send> (self=0x7f95625caff0, msg=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-channel-0.3.31/src/mpsc/mod.rs:816 -#3 0x00007f9580ae9b5a in tikv_util::worker::pool::{impl#12}::delay_notify::{closure#0} () at components/tikv_util/src/worker/pool.rs:462 -#4 0x00007f9580ade44f in futures_util::fns::{impl#0}::call_once, core::result::Result<(), tokio_timer::error::Error>, ()> (self=..., arg=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-util-0.3.31/src/fns.rs:15 -#5 0x00007f9580ad76a3 in futures_util::future::future::map::{impl#2}::poll, tikv_util::worker::pool::{impl#12}::delay_notify::{closure_env#0}, ()> (self=..., cx=0x7f95625cb260) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-util-0.3.31/src/future/future/map.rs:57 -#6 0x00007f9580ae0b75 in futures_util::future::future::{impl#15}::poll, tikv_util::worker::pool::{impl#12}::delay_notify::{closure_env#0}> (self=..., cx=0x7f95625cb260) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-util-0.3.31/src/lib.rs:86 -#7 0x00007f9582810696 in tikv_util::future::PollAtWake::poll (arc_self=0x7f95625cb3e8) at components/tikv_util/src/future.rs:156 -#8 0x00007f958281091b in tikv_util::future::{impl#3}::wake_by_ref (arc_self=0x7f95625cb3e8) at components/tikv_util/src/future.rs:200 -#9 0x00007f95828246cc in futures_task::waker::wake_by_ref_arc_raw (data=0x7f94bf062320) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-task-0.3.31/src/waker.rs:55 -#10 0x00007f9583825482 in core::task::wake::Waker::wake_by_ref (self=0x7f94bf0195e0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/task/wake.rs:300 -#11 0x00007f9583825ef3 in futures_util::compat::compat01as03::{impl#12}::notify (self=0x7f94bf0195e0) at src/compat/compat01as03.rs:339 -#12 0x00007f958385973b in futures::task_impl::NotifyHandle::notify (self=0x7f95625cb560, id=0) at src/task_impl/mod.rs:657 -#13 0x00007f9583856c98 in futures::task_impl::core::TaskUnpark::notify (self=0x7f95625cb558) at src/task_impl/core.rs:67 -#14 0x00007f958385387e in futures::task_impl::std::TaskUnpark::notify (self=0x7f95625cb558) at src/task_impl/std/mod.rs:204 -#15 0x00007f9583859711 in futures::task_impl::Task::notify (self=0x7f95625cb530) at src/task_impl/mod.rs:147 -#16 0x00007f9583859388 in futures::task_impl::atomic_task::AtomicTask::notify (self=0x7f94bf11e520) at src/task_impl/atomic_task.rs:250 -#17 0x00007f9582a00d83 in tokio_timer::timer::entry::Entry::fire (self=0x7f94bf11e480, when=57319452) at src/timer/entry.rs:251 -#18 0x00007f95828a5574 in tokio_timer::timer::Timer::process (self=0x7f95625cb7d8) at /home/xzx/.cargo/git/checkouts/tokio-8e927faba632ed16/4394380/tokio-timer/src/timer/mod.rs:276 -#19 0x00007f95828c99b7 in tokio_timer::timer::{impl#4}::park (self=0x7f95625cb7d8) at /home/xzx/.cargo/git/checkouts/tokio-8e927faba632ed16/4394380/tokio-timer/src/timer/mod.rs:379 -#20 0x00007f95828a52c9 in tokio_timer::timer::Timer::turn (self=0x7f95625cb7d8, max_wait=...) at /home/xzx/.cargo/git/checkouts/tokio-8e927faba632ed16/4394380/tokio-timer/src/timer/mod.rs:256 -#21 0x00007f958289e969 in tikv_util::timer::start_timer_thread::{closure#0} () at components/tikv_util/src/timer.rs:259 -#22 0x00007f95828de0e9 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0}, ()> () at components/tikv_util/src/sys/thread.rs:441 -#23 0x00007f95828ad307 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#24 0x00007f95828b811d in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#25 0x00007f958285ef41 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#26 0x00007f958286a875 in std::panicking::try::do_call, ()>, ()>>, ()> (data=0x7f95625cb9e0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#27 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#28 0x00007f9582869a6f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#29 0x00007f95828b7711 in std::panic::catch_unwind, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#30 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#31 0x00007f95827edc4f in core::ops::function::FnOnce::call_once, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#32 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#33 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#34 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#35 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#36 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 19 (Thread 0x7f9562dff640 (LWP 2739549) "tso-worker"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f9581da7762 in futures_executor::local_pool::run_executor::{closure#0}<(), futures_executor::local_pool::block_on::{closure_env#0}, (kvproto::protos::pdpb::TsoRequest, grpcio::call::WriteFlags), pd_client::errors::Error>, futures_util::stream::try_stream::ErrInto, pd_client::errors::Error>>>> (thread_notify=0x7f9562dfb8d8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-executor-0.3.15/src/local_pool.rs:99 -#5 0x00007f9581d5f70b in std::thread::local::LocalKey>::try_with, futures_executor::local_pool::run_executor::{closure_env#0}<(), futures_executor::local_pool::block_on::{closure_env#0}, (kvproto::protos::pdpb::TsoRequest, grpcio::call::WriteFlags), pd_client::errors::Error>, futures_util::stream::try_stream::ErrInto, pd_client::errors::Error>>>>, ()> (self=0x7f9584008e30, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#6 0x00007f9581d5f5d4 in std::thread::local::LocalKey>::with, futures_executor::local_pool::run_executor::{closure_env#0}<(), futures_executor::local_pool::block_on::{closure_env#0}, (kvproto::protos::pdpb::TsoRequest, grpcio::call::WriteFlags), pd_client::errors::Error>, futures_util::stream::try_stream::ErrInto, pd_client::errors::Error>>>>, ()> (self=0x7f9584008e30, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:246 -#7 0x00007f9581da7129 in futures_executor::local_pool::run_executor<(), futures_executor::local_pool::block_on::{closure_env#0}, (kvproto::protos::pdpb::TsoRequest, grpcio::call::WriteFlags), pd_client::errors::Error>, futures_util::stream::try_stream::ErrInto, pd_client::errors::Error>>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-executor-0.3.15/src/local_pool.rs:86 -#8 0x00007f9581da78e6 in futures_executor::local_pool::block_on, (kvproto::protos::pdpb::TsoRequest, grpcio::call::WriteFlags), pd_client::errors::Error>, futures_util::stream::try_stream::ErrInto, pd_client::errors::Error>>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-executor-0.3.15/src/local_pool.rs:315 -#9 0x00007f9581dfff48 in pd_client::tso::{impl#0}::new::{closure#0} () at components/pd_client/src/tso.rs:68 -#10 0x00007f9581e224e5 in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0} () at components/tikv_util/src/sys/thread.rs:441 -#11 0x00007f9581dab447 in std::sys_common::backtrace::__rust_begin_short_backtrace, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#12 0x00007f9581dac6c0 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#13 0x00007f9581e07944 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#14 0x00007f9581dea744 in std::panicking::try::do_call, ()>>, ()> (data=0x7f9562dcb560) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#15 0x00007f9581e0538b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#16 0x00007f9581dea6a8 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#17 0x00007f9581dac4e0 in std::panic::catch_unwind, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#18 std::thread::{impl#0}::spawn_unchecked_::{closure#1}, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#19 0x00007f9581e0a43f in core::ops::function::FnOnce::call_once, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#20 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#22 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#23 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#24 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 18 (Thread 0x7f95639ff640 (LWP 2739548) "pdmonitor-0"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f95639fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f95639fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f95639fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279713916976, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f95639cb648) at src/pool/spawn.rs:305 -#7 0x00007f95829b9e8b in yatp::pool::worker::WorkerThread::pop (self=0x7f95639cb648) at src/pool/worker.rs:36 -#8 0x00007f95829b9fbf in yatp::pool::worker::WorkerThread::run (self=...) at src/pool/worker.rs:44 -#9 0x00007f95829f5021 in yatp::pool::builder::{impl#3}::build::{closure#0}> () at src/pool/builder.rs:114 -#10 0x00007f95829aaef7 in std::sys_common::backtrace::__rust_begin_short_backtrace>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95829d9081 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f95829d4dc1 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f95829d9367 in std::panicking::try::do_call>, ()>>, ()> (data=0x7f95639cb860) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95829e4dab in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f95829d9152 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95829d8ea5 in std::panic::catch_unwind>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95829ad5ef in core::ops::function::FnOnce::call_once>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 17 (Thread 0x7f95645ff640 (LWP 2739547) "grpc-server-4"): -#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 -#1 0x00007f9583904b84 in pollset_work(grpc_pollset*, grpc_pollset_worker**, long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958395ff8a in cq_next(grpc_completion_queue*, gpr_timespec, void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f95837ecbeb in grpcio::cq::CompletionQueue::next (self=0x7f95645cb758) at src/cq.rs:202 -#4 0x00007f95837d06c5 in grpcio::env::poll_queue (tx=...) at src/env.rs:20 -#5 0x00007f95837d14bc in grpcio::env::{impl#0}::build::{closure#0} () at src/env.rs:107 -#6 0x00007f95837ae4e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#7 0x00007f95837c4791 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#8 0x00007f95837c3451 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#9 0x00007f95837e29e1 in std::panicking::try::do_call>, ()> (data=0x7f95645cb9b0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#10 0x00007f95837eed2b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#11 0x00007f95837e295f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#12 0x00007f95837c45b1 in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#14 0x00007f95837e329f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 16 (Thread 0x7f9564dff640 (LWP 2739546) "grpc-server-3"): -#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 -#1 0x00007f9583904b84 in pollset_work(grpc_pollset*, grpc_pollset_worker**, long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958395ff8a in cq_next(grpc_completion_queue*, gpr_timespec, void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f95837ecbeb in grpcio::cq::CompletionQueue::next (self=0x7f9564dcb758) at src/cq.rs:202 -#4 0x00007f95837d06c5 in grpcio::env::poll_queue (tx=...) at src/env.rs:20 -#5 0x00007f95837d14bc in grpcio::env::{impl#0}::build::{closure#0} () at src/env.rs:107 -#6 0x00007f95837ae4e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#7 0x00007f95837c4791 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#8 0x00007f95837c3451 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#9 0x00007f95837e29e1 in std::panicking::try::do_call>, ()> (data=0x7f9564dcb9b0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#10 0x00007f95837eed2b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#11 0x00007f95837e295f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#12 0x00007f95837c45b1 in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#14 0x00007f95837e329f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 15 (Thread 0x7f95657ff640 (LWP 2739545) "grpc-server-2"): -#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 -#1 0x00007f9583904b84 in pollset_work(grpc_pollset*, grpc_pollset_worker**, long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958395ff8a in cq_next(grpc_completion_queue*, gpr_timespec, void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f95837ecbeb in grpcio::cq::CompletionQueue::next (self=0x7f95657cb758) at src/cq.rs:202 -#4 0x00007f95837d06c5 in grpcio::env::poll_queue (tx=...) at src/env.rs:20 -#5 0x00007f95837d14bc in grpcio::env::{impl#0}::build::{closure#0} () at src/env.rs:107 -#6 0x00007f95837ae4e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#7 0x00007f95837c4791 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#8 0x00007f95837c3451 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#9 0x00007f95837e29e1 in std::panicking::try::do_call>, ()> (data=0x7f95657cb9b0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#10 0x00007f95837eed2b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#11 0x00007f95837e295f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#12 0x00007f95837c45b1 in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#14 0x00007f95837e329f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 14 (Thread 0x7f9565dfb640 (LWP 2739544) "grpc-server-1"): -#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 -#1 0x00007f9583904b84 in pollset_work(grpc_pollset*, grpc_pollset_worker**, long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958395ff8a in cq_next(grpc_completion_queue*, gpr_timespec, void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f95837ecbeb in grpcio::cq::CompletionQueue::next (self=0x7f9565dc7758) at src/cq.rs:202 -#4 0x00007f95837d06c5 in grpcio::env::poll_queue (tx=...) at src/env.rs:20 -#5 0x00007f95837d14bc in grpcio::env::{impl#0}::build::{closure#0} () at src/env.rs:107 -#6 0x00007f95837ae4e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#7 0x00007f95837c4791 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#8 0x00007f95837c3451 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#9 0x00007f95837e29e1 in std::panicking::try::do_call>, ()> (data=0x7f9565dc79b0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#10 0x00007f95837eed2b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#11 0x00007f95837e295f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#12 0x00007f95837c45b1 in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#14 0x00007f95837e329f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 13 (Thread 0x7f9565ffc640 (LWP 2739543) "grpc-server-0"): -#0 0x00007f957070e84e in epoll_wait () from /lib64/libc.so.6 -#1 0x00007f9583904b84 in pollset_work(grpc_pollset*, grpc_pollset_worker**, long) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958395ff8a in cq_next(grpc_completion_queue*, gpr_timespec, void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f95837ecbeb in grpcio::cq::CompletionQueue::next (self=0x7f9565fc8758) at src/cq.rs:202 -#4 0x00007f95837d06c5 in grpcio::env::poll_queue (tx=...) at src/env.rs:20 -#5 0x00007f95837d14bc in grpcio::env::{impl#0}::build::{closure#0} () at src/env.rs:107 -#6 0x00007f95837ae4e6 in std::sys_common::backtrace::__rust_begin_short_backtrace (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#7 0x00007f95837c4791 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#8 0x00007f95837c3451 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#9 0x00007f95837e29e1 in std::panicking::try::do_call>, ()> (data=0x7f9565fc89b0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#10 0x00007f95837eed2b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#11 0x00007f95837e295f in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#12 0x00007f95837c45b1 in std::panic::catch_unwind>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#14 0x00007f95837e329f in core::ops::function::FnOnce::call_once, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 12 (Thread 0x7f95669fd640 (LWP 2739542) "grpc_global_tim"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f958386d4a9 in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9583867500 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f9583917710 in timer_thread(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 11 (Thread 0x7f95673fe640 (LWP 2739541) "resolver-execut"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f958386d4a9 in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9583867500 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f9583909ab3 in grpc_core::Executor::ThreadMain(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 10 (Thread 0x7f9567dff640 (LWP 2739540) "default-executo"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f958386d4a9 in absl::lts_20211102::synchronization_internal::Waiter::Wait(absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#2 0x00007f958386be88 in AbslInternalPerThreadSemWait_lts_20211102 () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#3 0x00007f9583869d55 in absl::lts_20211102::CondVar::WaitCommon(absl::lts_20211102::Mutex*, absl::lts_20211102::synchronization_internal::KernelTimeout) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#4 0x00007f9583867500 in gpr_cv_wait () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#5 0x00007f9583909ab3 in grpc_core::Executor::ThreadMain(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#6 0x00007f958390a83d in grpc_core::(anonymous namespace)::ThreadInternalsPosix::ThreadInternalsPosix(char const*, void (*)(void*), void*, bool*, grpc_core::Thread::Options const&)::{lambda(void*)#1}::__invoke(void*) () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 9 (Thread 0x7f95685ff640 (LWP 2739539) "time-monitor"): -#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 -#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 -#2 0x00007f9583cea820 in std::sys::unix::thread::Thread::sleep () at library/std/src/sys/unix/thread.rs:243 -#3 std::thread::sleep () at library/std/src/thread/mod.rs:870 -#4 0x00007f958281b6ab in tikv_util::time::{impl#4}::new::{closure#0} std::time::SystemTime> () at components/tikv_util/src/time.rs:174 -#5 0x00007f95828ddfbe in tikv_util::sys::thread::{impl#1}::spawn_wrapper::{closure#0} std::time::SystemTime>, ()> () at components/tikv_util/src/sys/thread.rs:441 -#6 0x00007f95828ad317 in std::sys_common::backtrace::__rust_begin_short_backtrace std::time::SystemTime>, ()>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#7 0x00007f95828b8186 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0} std::time::SystemTime>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#8 0x00007f958285edb9 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0} std::time::SystemTime>, ()>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#9 0x00007f958286a97d in std::panicking::try::do_call std::time::SystemTime>, ()>, ()>>, ()> (data=0x7f95685cba50) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#10 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#11 0x00007f9582869b9e in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe std::time::SystemTime>, ()>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#12 0x00007f95828b6eaf in std::panic::catch_unwind std::time::SystemTime>, ()>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#13 std::thread::{impl#0}::spawn_unchecked_::{closure#1} std::time::SystemTime>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#14 0x00007f95827ed7ef in core::ops::function::FnOnce::call_once std::time::SystemTime>, ()>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#15 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#16 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#17 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#18 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#19 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 8 (Thread 0x7f9568d7f640 (LWP 2739538) "slogger"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f9583cea954 in std::sys::unix::futex::futex_wait () at library/std/src/sys/unix/futex.rs:62 -#2 std::sys_common::thread_parking::futex::Parker::park () at library/std/src/sys_common/thread_parking/futex.rs:52 -#3 std::thread::park () at library/std/src/thread/mod.rs:1066 -#4 0x00007f9582826c0b in crossbeam_channel::context::Context::wait_until (self=0x7f9568d491a8, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:162 -#5 0x00007f95828287c7 in crossbeam_channel::flavors::array::{impl#1}::recv::{closure#1} (cx=0x7f9568d491a8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/array.rs:432 -#6 0x00007f95827e5612 in crossbeam_channel::context::{impl#0}::with::{closure#0}, ()> (cx=0x7f9568d491a8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:52 -#7 0x00007f95827e5563 in crossbeam_channel::context::{impl#0}::with::{closure#1}, ()> (cell=0x7f9568d7b798) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:60 -#8 0x00007f95828b19cb in std::thread::local::LocalKey>>::try_with>, crossbeam_channel::context::{impl#0}::with::{closure_env#1}, ()>, ()> (self=0x7f958405cd10, f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/local.rs:270 -#9 0x00007f95827e527e in crossbeam_channel::context::Context::with, ()> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/context.rs:55 -#10 0x00007f95828286e5 in crossbeam_channel::flavors::array::Channel::recv (self=0x7f956a809c80, deadline=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/flavors/array.rs:421 -#11 0x00007f95828266ea in crossbeam_channel::channel::Receiver::recv (self=0x7f9568d496b8) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/crossbeam-channel-0.5.15/src/channel.rs:814 -#12 0x00007f957dfa3cff in slog_async::{impl#5}::spawn_thread::{closure#0}>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>> () at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/slog-async-2.6.0/lib.rs:286 -#13 0x00007f957e2c2927 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#14 0x00007f957e4446f4 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#15 0x00007f957e258494 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#16 0x00007f957e2c60e0 in std::panicking::try::do_call>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()>>, ()> (data=0x7f9568d4a1e0) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#17 0x00007f957e40507b in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#18 0x00007f957e2c3868 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#19 0x00007f957e444085 in std::panic::catch_unwind>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#20 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#21 0x00007f957e146bcf in core::ops::function::FnOnce::call_once>>, tikv_util::logger::RocksFormat>>, tikv_util::logger::TikvFormat>>, tikv_util::logger::RocksFormat>>>>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#22 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#23 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#24 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#25 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#26 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 7 (Thread 0x7f95697ff640 (LWP 2739537) "archive-worker-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f95697fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f95697fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f95697fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279713917136, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f95697cac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f95697cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f95697cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 6 (Thread 0x7f9569fff640 (LWP 2739536) "archive-worker-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f9569ffb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f9569ffb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f9569ffb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279713915216, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f9569fcac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f9569fcab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f9569fcb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 5 (Thread 0x7f956a7ff640 (LWP 2739535) "archive-worker-"): -#0 0x00007f957070770d in syscall () from /lib64/libc.so.6 -#1 0x00007f95829aae27 in parking_lot_core::thread_parker::imp::ThreadParker::futex_wait (self=0x7f956a7fb750, ts=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:112 -#2 0x00007f95829a4434 in parking_lot_core::thread_parker::imp::{impl#0}::park (self=0x7f956a7fb750) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/thread_parker/linux.rs:66 -#3 0x00007f95829aa8aa in parking_lot_core::parking_lot::park::{closure#0}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (thread_data=0x7f956a7fb730) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:635 -#4 0x00007f95829aa4f5 in parking_lot_core::parking_lot::with_thread_data, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}>> (f=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:207 -#5 parking_lot_core::parking_lot::park, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#1}, yatp::pool::spawn::{impl#10}::pop_or_sleep::{closure_env#2}> (key=140279713915056, validate=..., before_sleep=..., timed_out=..., park_token=..., timeout=...) at /home/xzx/.cargo/registry/src/index.crates.io-6f17d22bba15001f/parking_lot_core-0.9.1/src/parking_lot.rs:600 -#6 0x00007f95829b9b45 in yatp::pool::spawn::Local::pop_or_sleep (self=0x7f956a7cac70) at src/pool/spawn.rs:305 -#7 0x00007f95827fe875 in yatp::pool::worker::WorkerThread>::pop> (self=0x7f956a7cab70) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:36 -#8 0x00007f95827fe9b6 in yatp::pool::worker::WorkerThread>::run> (self=...) at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/worker.rs:44 -#9 0x00007f95827ff4e0 in yatp::pool::builder::{impl#3}::build::{closure#0}>> () at /home/xzx/.cargo/git/checkouts/yatp-e704b73c3ee279b6/793be4d/src/pool/builder.rs:114 -#10 0x00007f95828ad347 in std::sys_common::backtrace::__rust_begin_short_backtrace>>, ()> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/sys_common/backtrace.rs:155 -#11 0x00007f95828b8150 in std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure#0}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:529 -#12 0x00007f958285ee04 in core::panic::unwind_safe::{impl#23}::call_once<(), std::thread::{impl#0}::spawn_unchecked_::{closure#1}::{closure_env#0}>>, ()>> (self=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/panic/unwind_safe.rs:272 -#13 0x00007f958286a584 in std::panicking::try::do_call>>, ()>>, ()> (data=0x7f956a7cb170) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:552 -#14 0x00007f95828a27eb in __rust_try () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libtiflash_proxy.so -#15 0x00007f9582869c38 in std::panicking::try<(), core::panic::unwind_safe::AssertUnwindSafe>>, ()>>> (f=...) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panicking.rs:516 -#16 0x00007f95828b7f40 in std::panic::catch_unwind>>, ()>>, ()> (f=) at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/panic.rs:142 -#17 std::thread::{impl#0}::spawn_unchecked_::{closure#1}>>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/std/src/thread/mod.rs:528 -#18 0x00007f95827ed79f in core::ops::function::FnOnce::call_once>>, ()>, ()> () at /rustc/89e2160c4ca5808657ed55392620ed1dbbce78d1/library/core/src/ops/function.rs:250 -#19 0x00007f9583cff3b5 in alloc::boxed::{impl#47}::call_once<(), dyn core::ops::function::FnOnce<(), Output=()>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#20 alloc::boxed::{impl#47}::call_once<(), alloc::boxed::Box, alloc::alloc::Global>, alloc::alloc::Global> () at library/alloc/src/boxed.rs:2015 -#21 std::sys::unix::thread::{impl#2}::new::thread_start () at library/std/src/sys/unix/thread.rs:108 -#22 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#23 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 4 (Thread 0x7f956c1fe640 (LWP 2739532) "RaftStoreProxy"): -#0 0x00007f95706d4655 in clock_nanosleep@GLIBC_2.2.5 () from /lib64/libc.so.6 -#1 0x00007f95706d9247 in nanosleep () from /lib64/libc.so.6 -#2 0x00007f9583cea820 in std::sys::unix::thread::Thread::sleep () at library/std/src/sys/unix/thread.rs:243 -#3 std::thread::sleep () at library/std/src/thread/mod.rs:870 -#4 0x00007f957c39f4b3 in proxy_server::run::run_impl (config=..., proxy_config=..., engine_store_server_helper=0x7ffd1e7b8210) at proxy_components/proxy_server/src/run.rs:297 -#5 0x00007f957c3b3dc2 in proxy_server::run::run_tikv_proxy (config=..., proxy_config=..., engine_store_server_helper=0x7ffd1e7b8210) at proxy_components/proxy_server/src/run.rs:441 -#6 0x00007f957d4f6b4d in proxy_server::proxy::run_proxy (argc=15, argv=0x7f956a801000, engine_store_server_helper=0x7ffd1e7b8210) at proxy_components/proxy_server/src/proxy.rs:332 -#7 0x00007f957a6280e8 in raftstore_proxy::run_raftstore_proxy_ffi (argc=15, argv=0x7f956a801000, helper=0x7ffd1e7b8210) at raftstore-proxy/src/lib.rs:19 -#8 0x0000558c1ab7865c in DB::RaftStoreProxyRunner::runRaftStoreProxyFFI (pv=0x7f956f84ef10) at /DATA/disk3/xzx/tiflash/dbms/src/Storages/KVStore/ProxyStateMachine.h:244 -#9 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#10 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 3 (Thread 0x7f956cbff640 (LWP 2739500) "jemalloc_bg_thd"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f957068996f in pthread_cond_timedwait@@GLIBC_2.3.2 () from /lib64/libc.so.6 -#2 0x0000558c275f88b3 in background_thread_sleep (tsdn=0x7f956cbfba90, info=0x7f9570417480, interval=5000000000) at /DATA/disk3/xzx/tiflash/contrib/jemalloc/src/background_thread.c:159 -#3 0x0000558c275f841c in background_work_sleep_once (tsdn=0x7f956cbfba90, info=0x7f9570417480, ind=0) at /DATA/disk3/xzx/tiflash/contrib/jemalloc/src/background_thread.c:229 -#4 0x0000558c275f8018 in background_thread0_work (tsd=0x7f956cbfba90) at /DATA/disk3/xzx/tiflash/contrib/jemalloc/src/background_thread.c:374 -#5 0x0000558c275f7d9f in background_work (tsd=0x7f956cbfba90, ind=0) at /DATA/disk3/xzx/tiflash/contrib/jemalloc/src/background_thread.c:412 -#6 0x0000558c275f77ce in background_thread_entry (ind_arg=0x0) at /DATA/disk3/xzx/tiflash/contrib/jemalloc/src/background_thread.c:444 -#7 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#8 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 2 (Thread 0x7f956e5ff640 (LWP 2739499) "SignalListener"): -#0 0x00007f95706fdfcc in read () from /lib64/libc.so.6 -#1 0x0000558c1ac54b52 in DB::ReadBufferFromFileDescriptor::nextImpl (this=0x7f956e5cb290) at /DATA/disk3/xzx/tiflash/dbms/src/IO/Buffer/ReadBufferFromFileDescriptor.cpp:64 -#2 0x0000558c1aaa475f in DB::ReadBuffer::next (this=0x7f956e5cb290) at /DATA/disk3/xzx/tiflash/dbms/src/IO/Buffer/ReadBuffer.h:75 -#3 0x0000558c23425538 in DB::ReadBuffer::eof (this=0x7f956e5cb290) at /DATA/disk3/xzx/tiflash/dbms/src/IO/Buffer/ReadBuffer.h:100 -#4 SignalListener::run (this=0x7f95702f2ae0) at /DATA/disk3/xzx/tiflash/libs/libdaemon/src/BaseDaemon.cpp:294 -#5 0x0000558c27bf81ea in Poco::(anonymous namespace)::RunnableHolder::run (this=0x7f957020ef10) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread.cpp:43 -#6 0x0000558c27bf72ea in Poco::ThreadImpl::runnableEntry (pThread=0x7ffd1e7b9e58) at /DATA/disk3/xzx/tiflash/contrib/poco/Foundation/src/Thread_STD.cpp:139 -#7 0x0000558c27bf9760 in std::__1::__invoke[abi:ue170006](void* (*&&)(void*), Poco::ThreadImpl*&&) (__f=@0x7f95702f2b28: 0x558c27bf7280 , __args=@0x7f95702f2b30: 0x7ffd1e7b9e58) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__type_traits/invoke.h:340 -#8 0x0000558c27bf972e in std::__1::__thread_execute[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*, 2ul>(std::__1::tuple >, void* (*)(void*), Poco::ThreadImpl*>&, std::__1::__tuple_indices<2ul>) (__t=...) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:221 -#9 0x0000558c27bf9512 in std::__1::__thread_proxy[abi:ue170006] >, void* (*)(void*), Poco::ThreadImpl*> >(void*) (__vp=0x7f95702f2b20) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__thread/thread.h:232 -#10 0x00007f957068a19a in start_thread () from /lib64/libc.so.6 -#11 0x00007f957070f210 in clone3 () from /lib64/libc.so.6 -Thread 1 (Thread 0x7f95845be900 (LWP 2739406) "TiFlashMain"): -#0 0x00007f957068722a in __futex_abstimed_wait_common () from /lib64/libc.so.6 -#1 0x00007f957068bca4 in __pthread_clockjoin_ex () from /lib64/libc.so.6 -#2 0x00007f95847adbb3 in std::__1::thread::join() () from /DATA/disk3/xzx/tiup_deploy/dev/tiflash-7003/bin/tiflash/libc++.so.1 -#3 0x0000558c259b53fc in DB::TaskThreadPool::waitForStop (this=0x7f91cfae6dc0) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp:53 -#4 0x0000558c25988d5d in DB::TaskScheduler::~TaskScheduler (this=0x7f91cfae6d80) at /DATA/disk3/xzx/tiflash/dbms/src/Flash/Pipeline/Schedule/TaskScheduler.cpp:38 -#5 0x0000558c1ab8d2bb in std::__1::default_delete::operator()[abi:ue170006](DB::TaskScheduler*) const (this=0x558c29257c38 , __ptr=0x7f91cfae6d80) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__memory/unique_ptr.h:68 -#6 0x0000558c1ab8d1dc in std::__1::unique_ptr >::reset[abi:ue170006](DB::TaskScheduler*) (this=0x558c29257c38 , __p=0x0) at /DATA/disk3/xzx/llvm-17.0.6/bin/../include/c++/v1/__memory/unique_ptr.h:300 -#7 0x0000558c1ab5c549 in DB::Server::main(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&)::$_29::operator()() const (this=0x7ffd1e7b5ef0) at /DATA/disk3/xzx/tiflash/dbms/src/Server/Server.cpp:1207 -#8 0x0000558c1ab591c5 in ext::ScopeGuard, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&)::$_29>::~ScopeGuard() (this=0x7ffd1e7b5ef0) at /DATA/disk3/xzx/tiflash/libs/libcommon/include/ext/scope_guard.h:33 -#9 0x0000558c1ab5896f in DB::Server::main (this=0x7ffd1e7b9d08) at /DATA/disk3/xzx/tiflash/dbms/src/Server/Server.cpp:1260 -#10 0x0000558c27a41fc5 in Poco::Util::Application::run (this=0x7ffd1e7b9d08) at /DATA/disk3/xzx/tiflash/contrib/poco/Util/src/Application.cpp:335 -#11 0x0000558c27a60205 in Poco::Util::ServerApplication::run (this=0x7ffd1e7b9d08) at /DATA/disk3/xzx/tiflash/contrib/poco/Util/src/ServerApplication.cpp:94 -#12 0x0000558c1ab4cbc7 in DB::Server::run (this=0x7ffd1e7b9d08) at /DATA/disk3/xzx/tiflash/dbms/src/Server/Server.cpp:179 -#13 0x0000558c27a603e0 in Poco::Util::ServerApplication::run (this=0x7ffd1e7b9d08, argc=3, pArgv=0x7f95702f2980) at /DATA/disk3/xzx/tiflash/contrib/poco/Util/src/ServerApplication.cpp:618 -#14 0x0000558c1ab59338 in mainEntryClickHouseServer (argc=3, argv=0x7f95702f2980) at /DATA/disk3/xzx/tiflash/dbms/src/Server/Server.cpp:1280 -#15 0x0000558c1aa031a1 in main (argc_=4, argv_=0x7ffd1e7ba2a8) at /DATA/disk3/xzx/tiflash/dbms/src/Server/main.cpp:172 From 9bb484c00452814b6dc053d0d487795988017f5d Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 9 Jul 2025 19:27:08 +0800 Subject: [PATCH 091/118] fix bugs --- dbms/src/Operators/CTEPartition.cpp | 37 +++++++++++++++++++++++++---- dbms/src/Operators/CTEPartition.h | 28 ++++++++++++++++++++-- 2 files changed, 58 insertions(+), 7 deletions(-) diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 0a7b75e6293..81106a0b6cd 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -15,7 +15,6 @@ #include #include -#include #include #include #include @@ -24,8 +23,7 @@ namespace DB { size_t CTEPartition::getIdxInMemoryNoLock(size_t cte_reader_id) { - if (this->total_block_in_disk_num >= this->fetch_block_idxs[cte_reader_id]) - return this->fetch_block_idxs[cte_reader_id]; + RUNTIME_CHECK(this->fetch_block_idxs[cte_reader_id] >= this->total_block_in_disk_num); return this->fetch_block_idxs[cte_reader_id] - this->total_block_in_disk_num; } @@ -143,12 +141,31 @@ CTEOpStatus CTEPartition::spillBlocks() break; auto next_iter = std::next(split_iter); + auto * log = &Poco::Logger::get("LRUCache"); Blocks spilled_blocks; if (next_iter == split_idxs.end() || next_iter->second >= total_block_in_memory_num) + { + LOG_INFO( + log, + fmt::format( + "xzxdebug spill start logical: {}, {}~{}", + split_iter->first, + split_iter->second, + this->blocks.size() - 1)); spilled_blocks.assign(blocks_begin_iter + split_iter->second, this->blocks.end()); + } else + { + LOG_INFO( + log, + fmt::format( + "xzxdebug spill start logical: {}, {}~{}", + split_iter->first, + split_iter->second, + next_iter->second - 1)); spilled_blocks.assign(blocks_begin_iter + split_iter->second, blocks_begin_iter + next_iter->second); + } RUNTIME_CHECK(!spilled_blocks.empty()); @@ -204,7 +221,6 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) { auto spiller_iter = this->spillers.find(this->fetch_block_idxs[cte_reader_id]); if (spiller_iter == this->spillers.end()) - // All blocks in disk have been consumed return CTEOpStatus::OK; @@ -224,8 +240,19 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) retry = true; continue; } - + { + auto [iter, _] = this->fetch_idxs_disk.insert(std::make_pair(cte_reader_id, std::vector{})); + iter->second.push_back(this->fetch_block_idxs[cte_reader_id]); + } this->addIdxNoLock(cte_reader_id); + { + auto [iter, _] = this->total_fetch_disk_block_nums.insert(std::make_pair(cte_reader_id, 0)); + iter->second++; + } + { + auto [iter, _] = this->total_fetch_disk_row_nums.insert(std::make_pair(cte_reader_id, 0)); + iter->second += block.rows(); + } break; }; diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index d90eed36c53..7aadf9bbfee 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -62,19 +62,40 @@ struct CTEPartition for (const auto & item : this->total_fetch_row_nums) info_row = fmt::format("{} <{}: {}>", info_row, item.first, item.second); + String disk_info_block; + for (const auto & item : this->total_fetch_disk_block_nums) + disk_info_block = fmt::format("{} <{}: {}>", disk_info_block, item.first, item.second); + + String disk_info_row; + for (const auto & item : this->total_fetch_disk_row_nums) + disk_info_row = fmt::format("{} <{}: {}>", disk_info_row, item.first, item.second); + + String infos; + for (const auto & item : this->fetch_idxs_disk) + { + String nums; + for (auto idx : item.second) + nums = fmt::format("{} {}", nums, idx); + infos = fmt::format("{} ", infos, item.first, nums); + } + auto * log = &Poco::Logger::get("LRUCache"); LOG_INFO( log, fmt::format( "xzxdebug CTEPartition total_recv_block_num: {}, row: {}, total_spill_block_num: {}, " "total_fetch_block_num: {}, row num: {}, " - "total_byte_usage: {}", + "disk: {}, {}" + "total_byte_usage: {}, idxs_disk: {}", total_recv_block_num, total_recv_row_num, total_spill_block_num, info_block, info_row, - total_byte_usage)); + disk_info_block, + disk_info_row, + total_byte_usage, + infos)); } void init(std::shared_ptr spill_context_, size_t memory_threoshold_) @@ -120,6 +141,9 @@ struct CTEPartition size_t total_spill_block_num = 0; std::map total_fetch_block_nums; std::map total_fetch_row_nums; + std::map total_fetch_disk_block_nums; + std::map total_fetch_disk_row_nums; + std::map> fetch_idxs_disk; size_t total_byte_usage = 0; size_t partition_id; From 9b8a535e1aa54b6a4acd41f089274bda802aca66 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 14 Jul 2025 14:32:41 +0800 Subject: [PATCH 092/118] add debug logs --- dbms/src/Flash/Mpp/CTEManager.cpp | 11 +++++++++++ dbms/src/Operators/CTEPartition.cpp | 27 --------------------------- dbms/src/Operators/CTEPartition.h | 14 +++++++------- 3 files changed, 18 insertions(+), 34 deletions(-) diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index b5949a82cd1..4ab67569aa1 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -29,9 +29,15 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id) // Maybe the task is cancelled and the cte has been released return; + auto * log = &Poco::Logger::get("LRUCache"); + iter->second.sourceExit(); + LOG_INFO(log, fmt::format("xzxdebug total exit: {}, expect: {}", iter->second.getTotalExitNum(), iter->second.getExpectedTotalNum())); if (iter->second.getTotalExitNum() == iter->second.getExpectedTotalNum()) + { + LOG_INFO(log, "xzxdebug erase"); this->ctes.erase(iter); + } } void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) @@ -41,14 +47,19 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin if unlikely (iter == this->ctes.end()) // Maybe the task is cancelled and the cte has been released return; + auto * log = &Poco::Logger::get("LRUCache"); CTEWithCounter & cte_with_counter = iter->second; cte_with_counter.getCTE()->addResp(resp); cte_with_counter.sinkExit(); + LOG_INFO(log, fmt::format("xzxdebug total sink: {}, expect sink: {}, total exit: {}, expect: {}", cte_with_counter.getSinkExitNum(), cte_with_counter.getExpectedSinkNum(), cte_with_counter.getTotalExitNum(), cte_with_counter.getExpectedTotalNum())); if (cte_with_counter.getSinkExitNum() == cte_with_counter.getExpectedSinkNum()) cte_with_counter.getCTE()->notifyEOF(); if (cte_with_counter.getTotalExitNum() == cte_with_counter.getExpectedTotalNum()) + { + LOG_INFO(log, "xzxdebug erase"); this->ctes.erase(iter); + } } void CTEManager::releaseCTE(const String & query_id_and_cte_id) diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 81106a0b6cd..2d934970f0f 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -141,31 +141,12 @@ CTEOpStatus CTEPartition::spillBlocks() break; auto next_iter = std::next(split_iter); - auto * log = &Poco::Logger::get("LRUCache"); Blocks spilled_blocks; if (next_iter == split_idxs.end() || next_iter->second >= total_block_in_memory_num) - { - LOG_INFO( - log, - fmt::format( - "xzxdebug spill start logical: {}, {}~{}", - split_iter->first, - split_iter->second, - this->blocks.size() - 1)); spilled_blocks.assign(blocks_begin_iter + split_iter->second, this->blocks.end()); - } else - { - LOG_INFO( - log, - fmt::format( - "xzxdebug spill start logical: {}, {}~{}", - split_iter->first, - split_iter->second, - next_iter->second - 1)); spilled_blocks.assign(blocks_begin_iter + split_iter->second, blocks_begin_iter + next_iter->second); - } RUNTIME_CHECK(!spilled_blocks.empty()); @@ -180,14 +161,6 @@ CTEOpStatus CTEPartition::spillBlocks() split_iter++; } - LOG_INFO( - this->spill_context->getLog(), - fmt::format( - "Partition {} finishes cte spill for {}, spilled memory: {}", - this->partition_id, - this->spill_context->getQueryIdAndCTEId(), - this->memory_usage)); - this->blocks.clear(); this->memory_usage = 0; diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 7aadf9bbfee..f9b68418c2a 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -71,13 +71,13 @@ struct CTEPartition disk_info_row = fmt::format("{} <{}: {}>", disk_info_row, item.first, item.second); String infos; - for (const auto & item : this->fetch_idxs_disk) - { - String nums; - for (auto idx : item.second) - nums = fmt::format("{} {}", nums, idx); - infos = fmt::format("{} ", infos, item.first, nums); - } + // for (const auto & item : this->fetch_idxs_disk) + // { + // String nums; + // for (auto idx : item.second) + // nums = fmt::format("{} {}", nums, idx); + // infos = fmt::format("{} ", infos, item.first, nums); + // } auto * log = &Poco::Logger::get("LRUCache"); LOG_INFO( From b556b0951226e8dc1b917850dd3422a58d7a640c Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 15 Jul 2025 11:05:08 +0800 Subject: [PATCH 093/118] tweaking --- dbms/src/Flash/Coprocessor/DAGContext.h | 16 ++++++++++++++-- dbms/src/Flash/Mpp/CTEManager.cpp | 18 ++++++++++++++++++ dbms/src/Flash/Mpp/MPPTask.cpp | 7 ++++++- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 4 ++++ 4 files changed, 42 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index eef593b116e..e2ac0a7a086 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -14,6 +14,7 @@ #pragma once +#include #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" #ifdef __clang__ @@ -369,8 +370,17 @@ class DAGContext this->query_id_and_cte_id = query_id_and_cte_id; } - std::vector> getCTEs() const { return this->ctes; } - void addCTE(std::shared_ptr & cte) { this->ctes.push_back(cte); } + std::vector> getCTEs() + { + std::lock_guard lock(this->cte_mu); + return this->ctes; + } + + void addCTE(std::shared_ptr & cte) + { + std::lock_guard lock(this->cte_mu); + this->ctes.push_back(cte); + } public: DAGRequest dag_request; @@ -490,6 +500,8 @@ class DAGContext String connection_alias; String query_id_and_cte_id; + + std::mutex cte_mu; std::vector> ctes; }; diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index b5949a82cd1..a6c1d066cb4 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -29,9 +29,14 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id) // Maybe the task is cancelled and the cte has been released return; + auto * log = &Poco::Logger::get("LRUCache"); iter->second.sourceExit(); + LOG_INFO(log, fmt::format("xzxdebug total exit: {}, expect exit: {}", iter->second.getTotalExitNum(), iter->second.getExpectedTotalNum())); if (iter->second.getTotalExitNum() == iter->second.getExpectedTotalNum()) + { + LOG_INFO(log, fmt::format("xzxdebug cte {} is erased", query_id_and_cte_id)); this->ctes.erase(iter); + } } void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) @@ -45,10 +50,15 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin CTEWithCounter & cte_with_counter = iter->second; cte_with_counter.getCTE()->addResp(resp); cte_with_counter.sinkExit(); + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, fmt::format("xzxdebug sink: {}, expect sink: {} total exit: {}, expect exit: {}", cte_with_counter.getSinkExitNum(), cte_with_counter.getExpectedSinkNum(), cte_with_counter.getTotalExitNum(), cte_with_counter.getExpectedTotalNum())); if (cte_with_counter.getSinkExitNum() == cte_with_counter.getExpectedSinkNum()) cte_with_counter.getCTE()->notifyEOF(); if (cte_with_counter.getTotalExitNum() == cte_with_counter.getExpectedTotalNum()) + { + LOG_INFO(log, fmt::format("xzxdebug cte {} is erased", query_id_and_cte_id)); this->ctes.erase(iter); + } } void CTEManager::releaseCTE(const String & query_id_and_cte_id) @@ -56,7 +66,11 @@ void CTEManager::releaseCTE(const String & query_id_and_cte_id) std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if (iter != this->ctes.end()) + { + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, fmt::format("xzxdebug cte {} is erased", query_id_and_cte_id)); this->ctes.erase(iter); + } } std::shared_ptr CTEManager::getCTE( @@ -68,9 +82,13 @@ std::shared_ptr CTEManager::getCTE( std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if (iter == this->ctes.end()) + { this->ctes.insert(std::make_pair( query_id_and_cte_id, CTEWithCounter(std::make_shared(concurrency), expected_sink_num, expected_source_num))); + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, fmt::format("xzxdebug create cte {}", query_id_and_cte_id)); + } auto cte = this->ctes.find(query_id_and_cte_id)->second.getCTE(); cte->checkPartitionNum(concurrency); diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index b95bd1cdb2c..effa7779303 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -194,6 +194,8 @@ void MPPTask::abortCTE(const String & message) { if (this->has_cte_sink.load()) { + // auto * log = &Poco::Logger::get("LRUCache"); + auto ctes = this->dag_context->getCTEs(); // CTESource may be waiting for the finish signal from cte sink @@ -615,6 +617,9 @@ void MPPTask::runImpl() #endif auto result = query_executor_holder->execute(); + if (this->has_cte_sink.load()) + std::cout << "1"; + auto log_level = Poco::Message::PRIO_DEBUG; if (!result.is_success || status != RUNNING) log_level = Poco::Message::PRIO_INFORMATION; @@ -669,7 +674,7 @@ void MPPTask::runImpl() err_msg = err_msg.empty() ? catch_err_msg : fmt::format("{}, {}", err_msg, catch_err_msg); } - if (this->has_cte_sink.load() && !this->notify_cte_finish) + if unlikely (this->has_cte_sink.load() && !this->notify_cte_finish) { tipb::SelectResponse resp; this->context->getCTEManager()->releaseCTEBySink(resp, this->dag_context->getQueryIDAndCTEID()); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 5e994229c76..82a90b6985a 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -52,6 +52,8 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( Context & context, size_t concurrency) { + throw Exception("xzx test error in PhysicalCTESink"); + String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); exec_context.setQueryIDAndCTEID(query_id_and_cte_id); @@ -61,6 +63,8 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( this->expected_sink_num, this->expected_source_num); + exec_context.addCTE(cte); + RUNTIME_CHECK(group_builder.concurrency() <= concurrency); size_t id = 0; From ee6917c53e912f775ae450af629cfc43abbcbcc9 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 15 Jul 2025 17:46:09 +0800 Subject: [PATCH 094/118] refactor --- dbms/src/Flash/Coprocessor/DAGContext.h | 58 +++++++++++++++---- dbms/src/Flash/Executor/PipelineExecutor.cpp | 3 - .../Executor/PipelineExecutorContext.cpp | 10 +++- .../Flash/Executor/PipelineExecutorContext.h | 13 ----- dbms/src/Flash/FlashService.cpp | 2 +- dbms/src/Flash/Mpp/CTEManager.cpp | 21 ++++++- dbms/src/Flash/Mpp/MPPTask.cpp | 57 +++++++++++++----- dbms/src/Flash/Mpp/MPPTask.h | 1 + dbms/src/Flash/Planner/PhysicalPlan.cpp | 2 +- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 26 +++------ .../src/Flash/Planner/Plans/PhysicalCTESink.h | 15 +---- .../Flash/Planner/Plans/PhysicalCTESource.cpp | 24 ++------ .../Flash/Planner/Plans/PhysicalCTESource.h | 11 +--- dbms/src/Operators/CTE.cpp | 5 ++ dbms/src/Operators/CTE.h | 21 ++++++- dbms/src/Operators/CTEPartition.h | 3 +- dbms/src/Operators/CTEReader.cpp | 1 + dbms/src/Operators/CTEReader.h | 21 +++---- dbms/src/Operators/CTESourceOp.cpp | 3 + dbms/src/Operators/CTESourceOp.h | 20 ++++++- 20 files changed, 193 insertions(+), 124 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index e2ac0a7a086..aa1c00333b9 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -362,24 +362,58 @@ class DAGContext MPPReceiverSetPtr getMPPReceiverSet() const { return mpp_receiver_set; } - String getQueryIDAndCTEID() const noexcept { return this->query_id_and_cte_id; } - void setQueryIDAndCTEID(const String & query_id_and_cte_id) + String getQueryIDAndCTEIDForSink() { - // MPP Task has only one CTESink, it's impossible to set query_id_and_cte_id twice - RUNTIME_CHECK(this->query_id_and_cte_id.empty(), this->query_id_and_cte_id); - this->query_id_and_cte_id = query_id_and_cte_id; + std::lock_guard lock(this->cte_mu); + return this->query_id_and_cte_id_for_sink; + } + + String getQueryIDAndCTEIDForSource() + { + std::lock_guard lock(this->cte_mu); + return this->query_id_and_cte_id_for_source; + } + + void setQueryIDAndCTEIDForSink(const String & query_id_and_cte_id) + { + std::lock_guard lock(this->cte_mu); + + // MPP Task has only one CTESink, it's impossible to set query_id_and_cte_id_for_sink twice + RUNTIME_CHECK(this->query_id_and_cte_id_for_sink.empty(), this->query_id_and_cte_id_for_sink); + this->query_id_and_cte_id_for_sink = query_id_and_cte_id; + } + + void setQueryIDAndCTEIDForSource(const String & query_id_and_cte_id) + { + std::lock_guard lock(this->cte_mu); + + // MPP Task has only one CTESource, it's impossible to set query_id_and_cte_id_for_source twice + RUNTIME_CHECK(this->query_id_and_cte_id_for_source.empty(), this->query_id_and_cte_id_for_source); + this->query_id_and_cte_id_for_source = query_id_and_cte_id; + } + + std::shared_ptr getCTESink() + { + std::lock_guard lock(this->cte_mu); + return this->sink_cte; + } + + std::shared_ptr getCTESource() + { + std::lock_guard lock(this->cte_mu); + return this->source_cte; } - std::vector> getCTEs() + void setCTESink(std::shared_ptr & cte) { std::lock_guard lock(this->cte_mu); - return this->ctes; + this->sink_cte = cte; } - void addCTE(std::shared_ptr & cte) + void setCTESource(std::shared_ptr & cte) { std::lock_guard lock(this->cte_mu); - this->ctes.push_back(cte); + this->source_cte = cte; } public: @@ -499,10 +533,12 @@ class DAGContext // It's the session alias between mysql client and tidb String connection_alias; - String query_id_and_cte_id; + String query_id_and_cte_id_for_sink; + String query_id_and_cte_id_for_source; std::mutex cte_mu; - std::vector> ctes; + std::shared_ptr sink_cte; + std::shared_ptr source_cte; }; } // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutor.cpp b/dbms/src/Flash/Executor/PipelineExecutor.cpp index 332a1de4c19..af0f87e1565 100644 --- a/dbms/src/Flash/Executor/PipelineExecutor.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutor.cpp @@ -34,9 +34,6 @@ PipelineExecutor::PipelineExecutor( // For mpp task, there is a unique identifier MPPTaskId, so MPPTaskId is used here as the query id of PipelineExecutor. // But for cop/batchCop, there is no such unique identifier, so an empty value is given here, indicating that the query id of PipelineExecutor is invalid. /*query_id=*/context.getDAGContext()->isMPPTask() ? context.getDAGContext()->getMPPTaskId().toString() : "", - /*query_id_for_cte=*/context.getDAGContext()->isMPPTask() - ? context.getDAGContext()->getMPPTaskId().getQueryID() - : "", req_id, memory_tracker_, context.getDAGContext(), diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.cpp b/dbms/src/Flash/Executor/PipelineExecutorContext.cpp index f2a3609d6c2..8bb54adf5ec 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.cpp @@ -187,9 +187,13 @@ void PipelineExecutorContext::cancel() if (dag_context->tunnel_set) dag_context->tunnel_set->close(getTrimmedErrMsg(), false); - auto ctes = dag_context->getCTEs(); - if (!ctes.empty()) - ctes[0]->notifyCancel(getTrimmedErrMsg()); + auto cte = dag_context->getCTESink(); + if (cte) + cte->notifyCancel(getTrimmedErrMsg()); + + cte = dag_context->getCTESource(); + if (cte) + cte->notifyCancel(getTrimmedErrMsg()); if (auto mpp_receiver_set = dag_context->getMPPReceiverSet(); mpp_receiver_set) mpp_receiver_set->cancel(); diff --git a/dbms/src/Flash/Executor/PipelineExecutorContext.h b/dbms/src/Flash/Executor/PipelineExecutorContext.h index 0f39cfd5803..97fba40e889 100644 --- a/dbms/src/Flash/Executor/PipelineExecutorContext.h +++ b/dbms/src/Flash/Executor/PipelineExecutorContext.h @@ -56,7 +56,6 @@ class PipelineExecutorContext : private boost::noncopyable PipelineExecutorContext( const String & query_id_, - const String & query_id_for_cte_, const String & req_id, const MemoryTrackerPtr & mem_tracker_, DAGContext * dag_context_ = nullptr, @@ -65,7 +64,6 @@ class PipelineExecutorContext : private boost::noncopyable const KeyspaceID & keyspace_id_ = NullspaceID, const String & resource_group_name_ = "") : query_id(query_id_) - , query_id_for_cte(query_id_for_cte_) , log(Logger::get(req_id)) , mem_tracker(mem_tracker_) , dag_context(dag_context_) @@ -121,8 +119,6 @@ class PipelineExecutorContext : private boost::noncopyable const String & getQueryId() const { return query_id; } - const String & getQueryIdForCTE() const { return this->query_id_for_cte; } - const MemoryTrackerPtr & getMemoryTracker() const { return mem_tracker; } void triggerAutoSpill() const @@ -150,13 +146,6 @@ class PipelineExecutorContext : private boost::noncopyable void addOneTimeFuture(const OneTimeNotifyFuturePtr & future); - void setQueryIDAndCTEID(const String & query_id_and_cte_id) - { - this->dag_context->setQueryIDAndCTEID(query_id_and_cte_id); - } - - void addCTE(std::shared_ptr cte) { this->dag_context->addCTE(cte); } - private: bool setExceptionPtr(const std::exception_ptr & exception_ptr_); @@ -176,8 +165,6 @@ class PipelineExecutorContext : private boost::noncopyable private: const String query_id; - String query_id_for_cte; - LoggerPtr log; MemoryTrackerPtr mem_tracker; diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 07af1390649..8cc5cb6efd4 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -720,7 +720,7 @@ grpc::Status FlashService::CancelMPPTask( { CPUAffinityManager::getInstance().bindSelfGrpcThread(); // CancelMPPTask cancels the query of the task. - LOG_INFO(log, "cancel mpp task request: {}", request->DebugString()); + LOG_INFO(log, "xzxdebug cancel mpp task request: {}", request->DebugString()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index a6c1d066cb4..709ae7b02e7 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -31,7 +31,12 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id) auto * log = &Poco::Logger::get("LRUCache"); iter->second.sourceExit(); - LOG_INFO(log, fmt::format("xzxdebug total exit: {}, expect exit: {}", iter->second.getTotalExitNum(), iter->second.getExpectedTotalNum())); + LOG_INFO( + log, + fmt::format( + "xzxdebug total exit: {}, expect exit: {}", + iter->second.getTotalExitNum(), + iter->second.getExpectedTotalNum())); if (iter->second.getTotalExitNum() == iter->second.getExpectedTotalNum()) { LOG_INFO(log, fmt::format("xzxdebug cte {} is erased", query_id_and_cte_id)); @@ -51,7 +56,14 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin cte_with_counter.getCTE()->addResp(resp); cte_with_counter.sinkExit(); auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, fmt::format("xzxdebug sink: {}, expect sink: {} total exit: {}, expect exit: {}", cte_with_counter.getSinkExitNum(), cte_with_counter.getExpectedSinkNum(), cte_with_counter.getTotalExitNum(), cte_with_counter.getExpectedTotalNum())); + LOG_INFO( + log, + fmt::format( + "xzxdebug sink: {}, expect sink: {} total exit: {}, expect exit: {}", + cte_with_counter.getSinkExitNum(), + cte_with_counter.getExpectedSinkNum(), + cte_with_counter.getTotalExitNum(), + cte_with_counter.getExpectedTotalNum())); if (cte_with_counter.getSinkExitNum() == cte_with_counter.getExpectedSinkNum()) cte_with_counter.getCTE()->notifyEOF(); if (cte_with_counter.getTotalExitNum() == cte_with_counter.getExpectedTotalNum()) @@ -85,7 +97,10 @@ std::shared_ptr CTEManager::getCTE( { this->ctes.insert(std::make_pair( query_id_and_cte_id, - CTEWithCounter(std::make_shared(concurrency), expected_sink_num, expected_source_num))); + CTEWithCounter( + std::make_shared(concurrency, expected_sink_num), + expected_sink_num, + expected_source_num))); auto * log = &Poco::Logger::get("LRUCache"); LOG_INFO(log, fmt::format("xzxdebug create cte {}", query_id_and_cte_id)); } diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index effa7779303..0d765b4cf00 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -36,11 +36,11 @@ #include #include #include +#include #include #include #include -#include namespace DB { @@ -190,20 +190,24 @@ void MPPTask::abortQueryExecutor() } } +// CTESource may be waiting for the finish signal from cte sink +// We'd better to manually do notification in case of missing signal from cte sink +// or the CTESource will hang void MPPTask::abortCTE(const String & message) { + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, "xzxdebug enter cte abort, {} {}", this->has_cte_sink.load(), this->has_cte_source.load()); + if (this->has_cte_sink.load()) { - // auto * log = &Poco::Logger::get("LRUCache"); - - auto ctes = this->dag_context->getCTEs(); + this->dag_context->getCTESink()->notifyCancel(message); + this->context->getCTEManager()->releaseCTE(this->dag_context->getQueryIDAndCTEIDForSink()); + } - // CTESource may be waiting for the finish signal from cte sink - // We'd better to manually do notification in case of missing signal from cte sink - // or the CTESource will hang - for (auto & cte : ctes) - cte->notifyCancel(message); - this->context->getCTEManager()->releaseCTE(this->dag_context->getQueryIDAndCTEID()); + if (this->has_cte_source.load()) + { + this->dag_context->getCTESource()->notifyCancel(message); + this->context->getCTEManager()->releaseCTE(this->dag_context->getQueryIDAndCTEIDForSource()); } } @@ -218,7 +222,7 @@ void MPPTask::finishWrite() // The finish of pushing all blocks not means that cte sink job has been done. // Execution summary statistic also need to be sent. So we can release cte // only when execution sumary statistic has been sent. - this->context->getCTEManager()->releaseCTEBySink(resp, this->dag_context->getQueryIDAndCTEID()); + this->context->getCTEManager()->releaseCTEBySink(resp, this->dag_context->getQueryIDAndCTEIDForSink()); this->notify_cte_finish = true; } else @@ -243,8 +247,19 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) RUNTIME_CHECK_MSG( dag_context->dag_request.rootExecutor().has_cte_sink(), "Task should has either exchange sender or cte sink"); - // There is no need to register tunnel for cte sink + this->has_cte_sink.store(true); + const auto & cte_sink = dag_context->dag_request.rootExecutor().cte_sink(); + String query_id_and_cte_id + = fmt::format("{}_{}", context->getDAGContext()->getMPPTaskId().getQueryID(), cte_sink.cte_id()); + context->getDAGContext()->setQueryIDAndCTEIDForSink(query_id_and_cte_id); + auto cte = context->getCTEManager()->getCTE( + query_id_and_cte_id, + context->getMaxStreams(), + cte_sink.cte_sink_num(), + cte_sink.cte_source_num()); + cte->registerSink(); + context->getDAGContext()->setCTESink(cte); return; } auto tunnel_set_local = std::make_shared(log->identifier()); @@ -337,6 +352,20 @@ void MPPTask::initExchangeReceivers() throw Exception( "exchange receiver map can not be initialized, because the task is not in running state"); } + else if (executor.tp() == tipb::ExecType::TypeCTESource) + { + this->has_cte_source.store(true); + const auto & cte_source = executor.cte_source(); + String query_id_and_cte_id + = fmt::format("{}_{}", context->getDAGContext()->getMPPTaskId().getQueryID(), cte_source.cte_id()); + context->getDAGContext()->setQueryIDAndCTEIDForSource(query_id_and_cte_id); + auto cte = context->getCTEManager()->getCTE( + query_id_and_cte_id, + context->getMaxStreams(), + cte_source.cte_sink_num(), + cte_source.cte_source_num()); + context->getDAGContext()->setCTESource(cte); + } return true; }); } @@ -619,7 +648,7 @@ void MPPTask::runImpl() auto result = query_executor_holder->execute(); if (this->has_cte_sink.load()) std::cout << "1"; - + auto log_level = Poco::Message::PRIO_DEBUG; if (!result.is_success || status != RUNNING) log_level = Poco::Message::PRIO_INFORMATION; @@ -677,7 +706,7 @@ void MPPTask::runImpl() if unlikely (this->has_cte_sink.load() && !this->notify_cte_finish) { tipb::SelectResponse resp; - this->context->getCTEManager()->releaseCTEBySink(resp, this->dag_context->getQueryIDAndCTEID()); + this->context->getCTEManager()->releaseCTEBySink(resp, this->dag_context->getQueryIDAndCTEIDForSink()); this->notify_cte_finish = true; } diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index 512f68f50d5..84c281555d4 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -201,6 +201,7 @@ class MPPTask MPPReceiverSetPtr receiver_set; std::atomic has_cte_sink = false; + std::atomic has_cte_source = false; bool notify_cte_finish = false; int new_thread_count_of_mpp_receiver = 0; diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 3e38cff56a8..1aeac634df0 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -253,7 +253,7 @@ void PhysicalPlan::build(const tipb::Executor * executor) buildFinalProjectionForCTE(executor->cte_sink()); auto fine_grained_shuffle = FineGrainedShuffle(executor); GET_METRIC(tiflash_coprocessor_executor_count, type_cte_sink).Increment(); - pushBack(PhysicalCTESink::build(executor_id, log, fine_grained_shuffle, popBack(), executor->cte_sink())); + pushBack(PhysicalCTESink::build(executor_id, log, fine_grained_shuffle, popBack())); break; } default: diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 82a90b6985a..bc1c981dd29 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -20,7 +20,6 @@ #include #include -#include namespace DB { @@ -28,8 +27,7 @@ PhysicalPlanNodePtr PhysicalCTESink::build( const String & executor_id, const LoggerPtr & log, const FineGrainedShuffle & fine_grained_shuffle, - const PhysicalPlanNodePtr & child, - const ::tipb::CTESink & cte_sink) + const PhysicalPlanNodePtr & child) { RUNTIME_CHECK(child); @@ -38,10 +36,7 @@ PhysicalPlanNodePtr PhysicalCTESink::build( child->getSchema(), fine_grained_shuffle, log->identifier(), - child, - cte_sink.cte_id(), - cte_sink.cte_sink_num(), - cte_sink.cte_source_num()); + child); physical_cte_sink->disableRestoreConcurrency(); return physical_cte_sink; } @@ -52,21 +47,14 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( Context & context, size_t concurrency) { - throw Exception("xzx test error in PhysicalCTESink"); - - String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); - exec_context.setQueryIDAndCTEID(query_id_and_cte_id); - - std::shared_ptr cte = context.getCTEManager()->getCTE( - query_id_and_cte_id, - concurrency, - this->expected_sink_num, - this->expected_source_num); - - exec_context.addCTE(cte); + // throw Exception("xzxdebug test error in PhysicalCTESink"); // TODO remove it + // Partition number in CTE is equal to concurrency, we need to ensure that `group_builder.concurrency() <= concurrency` + // or some blocks in partition will not be fetched. RUNTIME_CHECK(group_builder.concurrency() <= concurrency); + std::shared_ptr cte = context.getDAGContext()->getCTESource(); + size_t id = 0; group_builder.transform([&](auto & builder) { builder.setSinkOp(std::make_unique(exec_context, log->identifier(), cte, id)); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h index 76126975d39..75376aafe95 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.h @@ -28,22 +28,15 @@ class PhysicalCTESink : public PhysicalUnary const String & executor_id, const LoggerPtr & log, const FineGrainedShuffle & fine_grained_shuffle, - const PhysicalPlanNodePtr & child, - const ::tipb::CTESink & cte_sink); + const PhysicalPlanNodePtr & child); PhysicalCTESink( const String & executor_id_, const NamesAndTypes & schema_, const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, - const PhysicalPlanNodePtr & child_, - UInt32 cte_id_, - Int32 expected_sink_num_, - Int32 expected_source_num_) + const PhysicalPlanNodePtr & child_) : PhysicalUnary(executor_id_, PlanType::CTESink, schema_, fine_grained_shuffle_, req_id, child_) - , cte_id(cte_id_) - , expected_sink_num(expected_sink_num_) - , expected_source_num(expected_source_num_) {} void finalizeImpl(const Names & parent_require) override; @@ -56,9 +49,5 @@ class PhysicalCTESink : public PhysicalUnary PipelineExecGroupBuilder & group_builder, Context & context, size_t /*concurrency*/) override; - - UInt32 cte_id; - Int32 expected_sink_num; - Int32 expected_source_num; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp index f9c8cac9677..f0e10f6a8a6 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.cpp @@ -22,7 +22,6 @@ #include #include -#include namespace DB { @@ -47,10 +46,7 @@ PhysicalPlanNodePtr PhysicalCTESource::build( schema, fine_grained_shuffle, log->identifier(), - Block(schema), - cte_source.cte_id(), - cte_source.cte_sink_num(), - cte_source.cte_source_num()); + Block(schema)); } void PhysicalCTESource::buildPipelineExecGroupImpl( @@ -59,23 +55,11 @@ void PhysicalCTESource::buildPipelineExecGroupImpl( Context & context, size_t concurrency) { - String query_id_and_cte_id = fmt::format("{}_{}", exec_context.getQueryIdForCTE(), this->cte_id); - exec_context.setQueryIDAndCTEID(query_id_and_cte_id); - - RUNTIME_CHECK(group_builder.concurrency() <= concurrency); - - auto cte_reader = std::make_shared( - query_id_and_cte_id, - concurrency, - context.getCTEManager(), - this->expected_sink_num, - this->expected_source_num); - exec_context.addCTE(cte_reader->getCTE()); + const String & query_id_and_cte_id = context.getDAGContext()->getQueryIDAndCTEIDForSource(); + auto cte_reader = std::make_shared(context); for (size_t i = 0; i < concurrency; ++i) - { group_builder.addConcurrency( - std::make_unique(exec_context, log->identifier(), cte_reader, i, schema)); - } + std::make_unique(exec_context, log->identifier(), cte_reader, i, schema, query_id_and_cte_id)); context.getDAGContext()->addInboundIOProfileInfos(this->executor_id, group_builder.getCurIOProfileInfos()); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h index e2e17d29ce2..514fbe6ba83 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESource.h @@ -36,15 +36,9 @@ class PhysicalCTESource : public PhysicalLeaf const NamesAndTypes & schema_, const FineGrainedShuffle & fine_grained_shuffle, const String & req_id, - const Block & sample_block_, - UInt64 cte_id_, - Int32 expected_sink_num_, - Int32 expected_source_num_) + const Block & sample_block_) : PhysicalLeaf(executor_id_, PlanType::CTESource, schema_, fine_grained_shuffle, req_id) , sample_block(sample_block_) - , cte_id(cte_id_) - , expected_sink_num(expected_sink_num_) - , expected_source_num(expected_source_num_) {} void finalizeImpl(const Names & parent_require) override; @@ -61,10 +55,7 @@ class PhysicalCTESource : public PhysicalLeaf private: Block sample_block; String query_id_and_cte_id; - UInt64 cte_id; std::vector partition_col_ids; TiDB::TiDBCollators partition_col_collators; - Int32 expected_sink_num; - Int32 expected_source_num; }; } // namespace DB diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 3bc4cd25440..209622b5ce6 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include @@ -24,6 +25,10 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block { std::shared_lock rw_lock(this->rw_lock); std::lock_guard lock(*this->partitions[partition_id].mu); + + if unlikely (!this->areAllSinksRegistered()) + return CTEOpStatus::SINK_NOT_REGISTERED; + auto status = this->checkBlockAvailableNoLock(cte_reader_id, partition_id); if (status != CTEOpStatus::OK) return status; diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 8253531029d..7e88ec85c48 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -30,8 +30,9 @@ namespace DB class CTE { public: - explicit CTE(size_t partition_num_) + explicit CTE(size_t partition_num_, size_t expected_sink_num_) : partition_num(partition_num_) + , expected_sink_num(expected_sink_num_) { for (size_t i = 0; i < this->partition_num; i++) { @@ -96,6 +97,21 @@ class CTE } } + void registerSink() + { + std::unique_lock lock(this->rw_lock); + this->registered_sink_num++; + } + + template + bool areAllSinksRegistered() + { + std::shared_lock lock(this->rw_lock, std::defer_lock); + if constexpr (need_lock) + lock.lock(); + return this->registered_sink_num == this->expected_sink_num; + } + private: CTEOpStatus checkBlockAvailableNoLock(size_t cte_reader_id, size_t partition_id) { @@ -137,5 +153,8 @@ class CTE tipb::SelectResponse resp; String err_msg; + + const size_t expected_sink_num; + size_t registered_sink_num = 0; }; } // namespace DB diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 5e9436f12ef..f1bec84ab76 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -26,7 +26,8 @@ enum class CTEOpStatus OK, BLOCK_NOT_AVAILABLE, END_OF_FILE, - CANCELLED + CANCELLED, + SINK_NOT_REGISTERED }; struct CTEPartition diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index d39f1f3c2a4..a8b1b203995 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -30,6 +30,7 @@ CTEOpStatus CTEReader::fetchNextBlock(size_t source_id, Block & block) if (this->resp.execution_summaries_size() == 0) this->cte->tryToGetResp(this->resp); } + case CTEOpStatus::SINK_NOT_REGISTERED: case CTEOpStatus::BLOCK_NOT_AVAILABLE: case CTEOpStatus::OK: return ret; diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index dd59cf53ddf..2996931efd0 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -14,9 +14,11 @@ #pragma once +#include #include #include #include +#include #include #include @@ -28,17 +30,14 @@ namespace DB class CTEReader { public: - CTEReader( - const String & query_id_and_cte_id_, - size_t partition_num, - CTEManager * cte_manager_, - Int32 expected_sink_num_, - Int32 expected_source_num_) - : query_id_and_cte_id(query_id_and_cte_id_) - , cte_manager(cte_manager_) - , cte(cte_manager_->getCTE(query_id_and_cte_id_, partition_num, expected_sink_num_, expected_source_num_)) + explicit CTEReader(Context & context) + : query_id_and_cte_id(context.getDAGContext()->getQueryIDAndCTEIDForSource()) + , cte_manager(context.getCTEManager()) + , cte(context.getDAGContext()->getCTESource()) , cte_reader_id(this->cte->getCTEReaderID()) - {} + { + RUNTIME_CHECK(cte); + } ~CTEReader() { @@ -60,6 +59,8 @@ class CTEReader std::shared_ptr getCTE() const { return this->cte; } size_t getID() const { return this->cte_reader_id; } + bool areAllSinksRegistered() { return this->cte->areAllSinksRegistered(); } + private: String query_id_and_cte_id; CTEManager * cte_manager; diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index 9883b1980ab..6f6ac482c88 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -39,6 +39,9 @@ OperatorStatus CTESourceOp::readImpl(Block & block) case CTEOpStatus::BLOCK_NOT_AVAILABLE: DB::setNotifyFuture(&(this->notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; + case CTEOpStatus::SINK_NOT_REGISTERED: + this->sw.start(); + return OperatorStatus::WAITING; default: throw Exception("Should not reach here"); } diff --git a/dbms/src/Operators/CTESourceOp.h b/dbms/src/Operators/CTESourceOp.h index cdf22e1812d..84aaa791458 100644 --- a/dbms/src/Operators/CTESourceOp.h +++ b/dbms/src/Operators/CTESourceOp.h @@ -24,6 +24,8 @@ #include +#include "Common/Stopwatch.h" + namespace DB { class CTESourceNotifyFuture : public NotifyFuture @@ -54,12 +56,14 @@ class CTESourceOp : public SourceOp const String & req_id, std::shared_ptr cte_reader_, size_t id_, - const NamesAndTypes & schema) + const NamesAndTypes & schema, + const String & query_id_and_cte_id_) : SourceOp(exec_context_, req_id) , cte_reader(cte_reader_) , io_profile_info(IOProfileInfo::createForRemote(profile_info_ptr, 1)) , id(id_) , notifier(this->cte_reader->getCTE(), this->cte_reader->getID(), this->id) + , query_id_and_cte_id(query_id_and_cte_id_) { setHeader(Block(getColumnWithTypeAndName(schema))); } @@ -72,6 +76,18 @@ class CTESourceOp : public SourceOp OperatorStatus readImpl(Block & block) override; + OperatorStatus awaitImpl() override + { + if (this->cte_reader->areAllSinksRegistered()) + return OperatorStatus::HAS_OUTPUT; + + if (this->sw.elapsedSeconds() >= 10) + throw Exception(fmt::format( + "cte sink can't be registered for 10s, query_id_and_cte_id: {}", + this->query_id_and_cte_id)); + return OperatorStatus::WAITING; + } + private: std::shared_ptr cte_reader; uint64_t total_rows{}; @@ -79,5 +95,7 @@ class CTESourceOp : public SourceOp tipb::SelectResponse resp; size_t id; CTESourceNotifyFuture notifier; + Stopwatch sw; + String query_id_and_cte_id; }; } // namespace DB From 0f1c0f2305f6618b578a5a92aa3913e8747e1b5d Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 16 Jul 2025 10:54:04 +0800 Subject: [PATCH 095/118] fix exception --- dbms/src/Flash/FlashService.cpp | 2 +- dbms/src/Flash/Mpp/CTEManager.cpp | 30 ------------------- dbms/src/Flash/Mpp/MPPTask.cpp | 3 -- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 6 ++-- 4 files changed, 3 insertions(+), 38 deletions(-) diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 8cc5cb6efd4..07af1390649 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -720,7 +720,7 @@ grpc::Status FlashService::CancelMPPTask( { CPUAffinityManager::getInstance().bindSelfGrpcThread(); // CancelMPPTask cancels the query of the task. - LOG_INFO(log, "xzxdebug cancel mpp task request: {}", request->DebugString()); + LOG_INFO(log, "cancel mpp task request: {}", request->DebugString()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index 709ae7b02e7..06c49f4209d 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -29,19 +29,9 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id) // Maybe the task is cancelled and the cte has been released return; - auto * log = &Poco::Logger::get("LRUCache"); iter->second.sourceExit(); - LOG_INFO( - log, - fmt::format( - "xzxdebug total exit: {}, expect exit: {}", - iter->second.getTotalExitNum(), - iter->second.getExpectedTotalNum())); if (iter->second.getTotalExitNum() == iter->second.getExpectedTotalNum()) - { - LOG_INFO(log, fmt::format("xzxdebug cte {} is erased", query_id_and_cte_id)); this->ctes.erase(iter); - } } void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) @@ -55,22 +45,10 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin CTEWithCounter & cte_with_counter = iter->second; cte_with_counter.getCTE()->addResp(resp); cte_with_counter.sinkExit(); - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO( - log, - fmt::format( - "xzxdebug sink: {}, expect sink: {} total exit: {}, expect exit: {}", - cte_with_counter.getSinkExitNum(), - cte_with_counter.getExpectedSinkNum(), - cte_with_counter.getTotalExitNum(), - cte_with_counter.getExpectedTotalNum())); if (cte_with_counter.getSinkExitNum() == cte_with_counter.getExpectedSinkNum()) cte_with_counter.getCTE()->notifyEOF(); if (cte_with_counter.getTotalExitNum() == cte_with_counter.getExpectedTotalNum()) - { - LOG_INFO(log, fmt::format("xzxdebug cte {} is erased", query_id_and_cte_id)); this->ctes.erase(iter); - } } void CTEManager::releaseCTE(const String & query_id_and_cte_id) @@ -78,11 +56,7 @@ void CTEManager::releaseCTE(const String & query_id_and_cte_id) std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if (iter != this->ctes.end()) - { - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, fmt::format("xzxdebug cte {} is erased", query_id_and_cte_id)); this->ctes.erase(iter); - } } std::shared_ptr CTEManager::getCTE( @@ -94,16 +68,12 @@ std::shared_ptr CTEManager::getCTE( std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if (iter == this->ctes.end()) - { this->ctes.insert(std::make_pair( query_id_and_cte_id, CTEWithCounter( std::make_shared(concurrency, expected_sink_num), expected_sink_num, expected_source_num))); - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, fmt::format("xzxdebug create cte {}", query_id_and_cte_id)); - } auto cte = this->ctes.find(query_id_and_cte_id)->second.getCTE(); cte->checkPartitionNum(concurrency); diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 0d765b4cf00..9869416c3b1 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -195,9 +195,6 @@ void MPPTask::abortQueryExecutor() // or the CTESource will hang void MPPTask::abortCTE(const String & message) { - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, "xzxdebug enter cte abort, {} {}", this->has_cte_sink.load(), this->has_cte_source.load()); - if (this->has_cte_sink.load()) { this->dag_context->getCTESink()->notifyCancel(message); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index bc1c981dd29..fc914dd0353 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -47,13 +47,12 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( Context & context, size_t concurrency) { - // throw Exception("xzxdebug test error in PhysicalCTESink"); // TODO remove it - // Partition number in CTE is equal to concurrency, we need to ensure that `group_builder.concurrency() <= concurrency` // or some blocks in partition will not be fetched. RUNTIME_CHECK(group_builder.concurrency() <= concurrency); - std::shared_ptr cte = context.getDAGContext()->getCTESource(); + std::shared_ptr cte = context.getDAGContext()->getCTESink(); + RUNTIME_CHECK(cte); size_t id = 0; group_builder.transform([&](auto & builder) { @@ -71,5 +70,4 @@ const Block & PhysicalCTESink::getSampleBlock() const { return child->getSampleBlock(); } - } // namespace DB From a11fc8cb0022389c31ea6b1085612b9d8fedb054 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 16 Jul 2025 11:01:13 +0800 Subject: [PATCH 096/118] tweaking --- dbms/src/Operators/CTESourceOp.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Operators/CTESourceOp.h b/dbms/src/Operators/CTESourceOp.h index 84aaa791458..1975fe88d41 100644 --- a/dbms/src/Operators/CTESourceOp.h +++ b/dbms/src/Operators/CTESourceOp.h @@ -21,11 +21,10 @@ #include #include #include +#include #include -#include "Common/Stopwatch.h" - namespace DB { class CTESourceNotifyFuture : public NotifyFuture From a5160a520dea24314875468a3f7596a13a9f334d Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 16 Jul 2025 11:03:27 +0800 Subject: [PATCH 097/118] fix --- dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp | 2 +- dbms/src/Operators/CTESourceOp.h | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 91eb13c8fa5..1664bbf8e31 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -75,7 +75,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( spill_config, group_builder.getCurrentHeader(), settings.max_bytes_before_cte_spill, - query_id_and_cte_id); + context.getDAGContext()->getQueryIDAndCTEIDForSink()); } void PhysicalCTESink::finalizeImpl(const Names & parent_require) diff --git a/dbms/src/Operators/CTESourceOp.h b/dbms/src/Operators/CTESourceOp.h index 357ea49885f..2115533ebb5 100644 --- a/dbms/src/Operators/CTESourceOp.h +++ b/dbms/src/Operators/CTESourceOp.h @@ -22,7 +22,6 @@ #include #include #include -#include #include From 8326817d621df92baaa7e373815f4ab8e1844a55 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 16 Jul 2025 15:11:29 +0800 Subject: [PATCH 098/118] remove useless codes --- dbms/src/Flash/Mpp/MPPTask.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 9869416c3b1..6a617706a44 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -643,8 +643,6 @@ void MPPTask::runImpl() #endif auto result = query_executor_holder->execute(); - if (this->has_cte_sink.load()) - std::cout << "1"; auto log_level = Poco::Message::PRIO_DEBUG; if (!result.is_success || status != RUNNING) From 7e8b7d67498b47a32c63b3f08497dadda002797b Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 16 Jul 2025 15:11:47 +0800 Subject: [PATCH 099/118] remove useless codes --- dbms/src/Flash/Mpp/MPPTask.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 9869416c3b1..6a617706a44 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -643,8 +643,6 @@ void MPPTask::runImpl() #endif auto result = query_executor_holder->execute(); - if (this->has_cte_sink.load()) - std::cout << "1"; auto log_level = Poco::Message::PRIO_DEBUG; if (!result.is_success || status != RUNNING) From 80dc7b4b12ab9ef31ec99351b115263755943cdc Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 16 Jul 2025 15:56:37 +0800 Subject: [PATCH 100/118] add log --- dbms/src/Operators/CTE.h | 2 ++ dbms/src/Operators/CTEReader.cpp | 4 ++++ dbms/src/Operators/CTEReader.h | 3 +++ 3 files changed, 9 insertions(+) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 15f8c3f5154..e1a9e9e1e1d 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -130,6 +130,8 @@ class CTE return this->registered_sink_num == this->expected_sink_num; } + LoggerPtr getLog() const { return this->cte_spill_context->getLog(); } + private: void notifyImpl(bool is_eof, const String & msg) { diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index d71655891bf..2b7f8b9c578 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -46,6 +46,10 @@ CTEOpStatus CTEReader::fetchNextBlock(size_t partition_id, Block & block) CTEOpStatus CTEReader::fetchBlockFromDisk(size_t partition_id, Block & block) { + bool expected_bool = false; + if (this->is_first_log.compare_exchange_weak(expected_bool, true)) + LOG_INFO(this->cte->getLog(), "Begin restore data from disk for cte."); + return this->cte->getBlockFromDisk(this->cte_reader_id, partition_id, block); } } // namespace DB diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index bc4873aa145..6e7e53ef5ad 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -22,6 +22,7 @@ #include #include +#include #include #include @@ -71,5 +72,7 @@ class CTEReader std::mutex mu; bool resp_fetched = false; tipb::SelectResponse resp; + + std::atomic_bool is_first_log = false; }; } // namespace DB From f091e35339e06580fe5ed36e39ab0898ad5b12b2 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 18 Jul 2025 11:43:54 +0800 Subject: [PATCH 101/118] support auto spill --- dbms/src/Flash/FlashService.cpp | 6 ++ .../Flash/Planner/Plans/PhysicalCTESink.cpp | 2 +- dbms/src/Interpreters/CTESpillContext.cpp | 16 +++++ dbms/src/Interpreters/CTESpillContext.h | 67 +++++++++++++++++-- dbms/src/Operators/CTE.cpp | 29 ++++---- dbms/src/Operators/CTE.h | 8 +-- dbms/src/Operators/CTEPartition.cpp | 31 ++++----- dbms/src/Operators/CTEPartition.h | 43 +++++------- 8 files changed, 137 insertions(+), 65 deletions(-) diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 07af1390649..2e3ce0b2ef8 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -151,6 +151,7 @@ void updateSettingsFromTiDB(const grpc::ServerContext * grpc_context, ContextPtr std::make_pair("tidb_max_bytes_before_tiflash_external_join", "max_bytes_before_external_join"), std::make_pair("tidb_max_bytes_before_tiflash_external_group_by", "max_bytes_before_external_group_by"), std::make_pair("tidb_max_bytes_before_tiflash_external_sort", "max_bytes_before_external_sort"), + std::make_pair("tidb_max_bytes_before_tiflash_cte_spill", "max_bytes_before_cte_spill"), std::make_pair("tiflash_mem_quota_query_per_node", "max_memory_usage"), std::make_pair("tiflash_query_spill_ratio", "auto_memory_revoke_trigger_threshold"), std::make_pair("tiflash_use_hash_join_v2", "enable_hash_join_v2"), @@ -188,6 +189,11 @@ void updateSettingsForAutoSpill(ContextPtr & context, const LoggerPtr & log) need_log_warning = true; context->setSetting("max_bytes_before_external_join", "0"); } + if (context->getSettingsRef().max_bytes_before_cte_spill > 0) + { + need_log_warning = true; + context->setSetting("max_bytes_before_cte_spill", "0"); + } if (need_log_warning) LOG_WARNING(log, "auto spill is enabled, so per operator's memory threshold is disabled"); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 1664bbf8e31..9a007ecc610 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -75,7 +75,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( spill_config, group_builder.getCurrentHeader(), settings.max_bytes_before_cte_spill, - context.getDAGContext()->getQueryIDAndCTEIDForSink()); + context); } void PhysicalCTESink::finalizeImpl(const Names & parent_require) diff --git a/dbms/src/Interpreters/CTESpillContext.cpp b/dbms/src/Interpreters/CTESpillContext.cpp index 07a28498854..adbf3384b7a 100644 --- a/dbms/src/Interpreters/CTESpillContext.cpp +++ b/dbms/src/Interpreters/CTESpillContext.cpp @@ -33,4 +33,20 @@ SpillerPtr CTESpillContext::getSpiller(size_t partition_id, size_t spill_id) return std::make_unique(config, false, this->partition_num, this->spill_block_schema, this->log, 1, false); } + +Int64 CTESpillContext::triggerSpillImpl(Int64 expected_released_memories) +{ + for (size_t i = 0; i < this->partition_num; i++) + { + std::unique_lock lock(this->aux_locks[i]); + if (this->statuses[i] != CTEPartitionStatus::NORMAL) + continue; + + this->statuses[i] = CTEPartitionStatus::NEED_SPILL; + expected_released_memories = std::max(expected_released_memories - this->memory_usages[i], 0); + if (expected_released_memories <= 0) + return expected_released_memories; + } + return expected_released_memories; +} } // namespace DB diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h index ca8436420a2..6fdca7658f1 100644 --- a/dbms/src/Interpreters/CTESpillContext.h +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -21,33 +21,90 @@ #include #include +#include + namespace DB { -class CTESpillContext +enum CTEPartitionStatus +{ + NORMAL = 0, + NEED_SPILL, + IN_SPILLING, +}; + +class CTESpillContext final : public OperatorSpillContext { public: CTESpillContext( + UInt64 operator_spill_threshold_, size_t partition_num_, const SpillConfig & spill_config_, const Block & spill_block_schema_, const String & query_id_and_cte_id_) - : partition_num(partition_num_) + : OperatorSpillContext(operator_spill_threshold_, "cte", Logger::get(query_id_and_cte_id_)) + , partition_num(partition_num_) + , partition_memory_threoshold(operator_spill_threshold_ / partition_num_) + , memory_usages(partition_num_) , spill_config(spill_config_) , spill_block_schema(spill_block_schema_) , query_id_and_cte_id(query_id_and_cte_id_) - , log(Logger::get(query_id_and_cte_id_)) - {} + , aux_locks(partition_num) + , statuses(partition_num) + , tmp_blocks(partition_num) + { + for (auto & status : this->statuses) + status = CTEPartitionStatus::NORMAL; + } SpillerPtr getSpiller(size_t partition_id, size_t spill_id); LoggerPtr getLog() const { return this->log; } String getQueryIdAndCTEId() const { return this->query_id_and_cte_id; } + bool supportAutoTriggerSpill() const override { return true; } + Int64 triggerSpillImpl(Int64 expected_released_memories) override; + + std::mutex * getPartitionAuxMutex(size_t partition_id) { return &(this->aux_locks[partition_id]); } + + CTEPartitionStatus getPartitionStatusNoLock(size_t partition_id) const { return this->statuses[partition_id]; } + void setPartitionStatusNoLock(size_t partition_id, CTEPartitionStatus status) + { + this->statuses[partition_id] = status; + } + + void pushTmpBlock(size_t partition_id, const Block & block) { this->tmp_blocks[partition_id].push_back(block); } + Blocks & getTmpBlocks(size_t partition_id) { return this->tmp_blocks[partition_id]; } + + void clearMemoryUsage(size_t partition_id) { this->memory_usages[partition_id].store(0); } + void addMemoryUsage(size_t partition_id, size_t delta) { this->memory_usages[partition_id].fetch_add(delta); } + bool exceedMemoryThreshold(size_t partition_id) const + { + if (this->partition_memory_threoshold == 0) + return false; + return this->memory_usages[partition_id].load() > this->partition_memory_threoshold; + } + +protected: + Int64 getTotalRevocableMemoryImpl() override + { + Int64 total_memory = 0; + for (const auto & memory_usage : this->memory_usages) + total_memory += memory_usage.load(); + return total_memory; + } + private: size_t partition_num; + size_t partition_memory_threoshold; + std::vector memory_usages; + SpillConfig spill_config; Block spill_block_schema; String query_id_and_cte_id; - LoggerPtr log; + + // Protecting cte_status and tmp_blocks + std::vector aux_locks; + std::vector statuses; + std::vector tmp_blocks; }; } // namespace DB diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 976e3210e56..52550f33a85 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -28,20 +29,22 @@ namespace DB { void CTE::initCTESpillContext( - const SpillConfig & spill_config_, - const Block & spill_block_schema_, - UInt64 operator_spill_threshold_, - const String & query_id_and_cte_id) + const SpillConfig & spill_config, + const Block & spill_block_schema, + UInt64 operator_spill_threshold, + Context & context) { std::unique_lock lock(this->rw_lock); this->cte_spill_context = std::make_shared( + operator_spill_threshold, this->partition_num, - spill_config_, - spill_block_schema_, - query_id_and_cte_id); - size_t memory_threoshold = operator_spill_threshold_ / this->partition_num; + spill_config, + spill_block_schema, + context.getDAGContext()->getQueryIDAndCTEIDForSink()); for (auto & item : this->partitions) - item.init(this->cte_spill_context, memory_threoshold); + item.init(this->cte_spill_context); + + context.getDAGContext()->registerOperatorSpillContext(this->cte_spill_context); } CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block & block) @@ -106,8 +109,8 @@ void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_ return; } - std::lock_guard aux_lock(*(this->partitions[partition_id].aux_lock)); - if (this->partitions[partition_id].status == CTEPartitionStatus::IN_SPILLING) + std::lock_guard aux_lock(*(this->partitions[partition_id].getAuxMutex())); + if (this->partitions[partition_id].getStatusNoLock() == CTEPartitionStatus::IN_SPILLING) { this->notifyTaskDirectly(partition_id, std::move(task)); return; @@ -129,8 +132,8 @@ void CTE::checkInSpillingAndRegisterTask(TaskPtr && task, size_t partition_id) return; } - std::lock_guard aux_lock(*(this->partitions[partition_id].aux_lock)); - if (this->partitions[partition_id].status == CTEPartitionStatus::IN_SPILLING) + std::lock_guard aux_lock(*(this->partitions[partition_id].getAuxMutex())); + if (this->partitions[partition_id].getStatusNoLock() == CTEPartitionStatus::IN_SPILLING) this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); else this->notifyTaskDirectly(partition_id, std::move(task)); diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index e1a9e9e1e1d..a44a1eec7aa 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -47,10 +47,10 @@ class CTE } void initCTESpillContext( - const SpillConfig & spill_config_, - const Block & spill_block_schema_, - UInt64 operator_spill_threshold_, - const String & query_id_and_cte_id); + const SpillConfig & spill_config, + const Block & spill_block_schema, + UInt64 operator_spill_threshold, + Context & context); void checkPartitionNum(size_t partition_num) const { diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 2d934970f0f..330e2916039 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -29,8 +29,8 @@ size_t CTEPartition::getIdxInMemoryNoLock(size_t cte_reader_id) CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) { - std::lock_guard aux_lock(*(this->aux_lock)); - if (this->status == CTEPartitionStatus::IN_SPILLING) + std::lock_guard aux_lock(*(this->getAuxMutex())); + if (this->getStatusNoLock() == CTEPartitionStatus::IN_SPILLING) return CTEOpStatus::WAIT_SPILL; std::lock_guard lock(*this->mu); @@ -57,9 +57,9 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) CTEOpStatus CTEPartition::pushBlock(const Block & block) { - std::unique_lock aux_lock(*(this->aux_lock)); + std::unique_lock aux_lock(*(this->getAuxMutex())); CTEOpStatus ret_status = CTEOpStatus::OK; - switch (this->status) + switch (this->getStatusNoLock()) { case CTEPartitionStatus::NEED_SPILL: ret_status = CTEOpStatus::NEED_SPILL; @@ -67,7 +67,7 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) ret_status = CTEOpStatus::WAIT_SPILL; if likely (block.rows() != 0) // Block memory usage will be calculated after the finish of spill - this->tmp_blocks.push_back(block); + this->pushTmpBlock(block); return ret_status; case CTEPartitionStatus::NORMAL: break; @@ -81,7 +81,7 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) this->total_recv_row_num += block.rows(); this->total_byte_usage += block.bytes(); - this->memory_usage += block.bytes(); + this->addMemoryUsage(block.bytes()); this->blocks.push_back(block); this->pipe_cv->notifyOne(); @@ -103,8 +103,8 @@ CTEOpStatus CTEPartition::spillBlocks() this->spill_context->getQueryIdAndCTEId())); std::unique_lock lock(*(this->mu), std::defer_lock); { - std::lock_guard aux_lock(*(this->aux_lock)); - switch (this->status) + std::lock_guard aux_lock(*(this->getAuxMutex())); + switch (this->getStatusNoLock()) { case CTEPartitionStatus::NORMAL: return CTEOpStatus::OK; @@ -116,12 +116,13 @@ CTEOpStatus CTEPartition::spillBlocks() } lock.lock(); - for (const auto & block : this->tmp_blocks) + Blocks & tmp_blocks = this->getTmpBlocks(); + for (const auto & block : tmp_blocks) { - this->memory_usage += block.bytes(); + this->addMemoryUsage(block.bytes()); this->blocks.push_back(block); } - this->tmp_blocks.clear(); + tmp_blocks.clear(); } // Key represents logical index @@ -162,9 +163,9 @@ CTEOpStatus CTEPartition::spillBlocks() } this->blocks.clear(); - this->memory_usage = 0; + this->clearMemoryUsage(); - std::lock_guard aux_lock(*(this->aux_lock)); + std::lock_guard aux_lock(*(this->getAuxMutex())); this->setCTEPartitionStatusNoLock(CTEPartitionStatus::NORMAL); // Many tasks may be waiting for the finish of spill @@ -176,8 +177,8 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) { std::unique_lock lock(*(this->mu), std::defer_lock); { - std::lock_guard aux_lock(*(this->aux_lock)); - if (this->status == CTEPartitionStatus::IN_SPILLING) + std::lock_guard aux_lock(*(this->getAuxMutex())); + if (this->getStatusNoLock() == CTEPartitionStatus::IN_SPILLING) return CTEOpStatus::WAIT_SPILL; lock.lock(); diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index f456f8f3f1e..14c3b16beb5 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -25,13 +25,6 @@ namespace DB { -enum CTEPartitionStatus -{ - NORMAL = 0, - NEED_SPILL, - IN_SPILLING, -}; - enum class CTEOpStatus { OK, @@ -50,7 +43,6 @@ struct CTEPartition : partition_id(partition_id_) , mu(std::make_unique()) , pipe_cv(std::make_unique()) - , aux_lock(std::make_unique()) {} void debugOutput() @@ -99,11 +91,7 @@ struct CTEPartition infos)); } - void init(std::shared_ptr spill_context_, size_t memory_threoshold_) - { - this->spill_context = spill_context_; - this->memory_threoshold = memory_threoshold_; - } + void init(std::shared_ptr spill_context_) { this->spill_context = spill_context_; } size_t getIdxInMemoryNoLock(size_t cte_reader_id); bool isBlockAvailableInDiskNoLock(size_t cte_reader_id) @@ -114,15 +102,13 @@ struct CTEPartition { return this->getIdxInMemoryNoLock(cte_reader_id) < this->blocks.size(); } - void setCTEPartitionStatusNoLock(CTEPartitionStatus status) { this->status = status; } - bool isSpillTriggeredNoLock() const { return this->total_block_in_disk_num > 0; } - void addIdxNoLock(size_t cte_reader_id) { this->fetch_block_idxs[cte_reader_id]++; } - bool exceedMemoryThresholdNoLock() const + void setCTEPartitionStatusNoLock(CTEPartitionStatus status) const { - if (this->memory_threoshold == 0) - return false; - return this->memory_usage > this->memory_threoshold; + this->spill_context->setPartitionStatusNoLock(this->partition_id, status); } + bool isSpillTriggeredNoLock() const { return this->total_block_in_disk_num > 0; } + void addIdxNoLock(size_t cte_reader_id) { this->fetch_block_idxs[cte_reader_id]++; } + bool exceedMemoryThresholdNoLock() const { return this->spill_context->exceedMemoryThreshold(this->partition_id); } CTEOpStatus pushBlock(const Block & block); CTEOpStatus tryGetBlock(size_t cte_reader_id, Block & block); @@ -137,6 +123,16 @@ struct CTEPartition return this->isBlockAvailableInMemoryNoLock(cte_reader_id); } + std::mutex * getAuxMutex() const { return this->spill_context->getPartitionAuxMutex(this->partition_id); } + CTEPartitionStatus getStatusNoLock() const + { + return this->spill_context->getPartitionStatusNoLock(this->partition_id); + } + void pushTmpBlock(const Block & block) const { this->spill_context->pushTmpBlock(this->partition_id, block); } + Blocks & getTmpBlocks() const { return this->spill_context->getTmpBlocks(this->partition_id); } + void addMemoryUsage(size_t delta) const { this->spill_context->addMemoryUsage(this->partition_id, delta); } + void clearMemoryUsage() const { this->spill_context->clearMemoryUsage(this->partition_id); } + size_t total_recv_block_num = 0; size_t total_recv_row_num = 0; size_t total_spill_block_num = 0; @@ -152,15 +148,8 @@ struct CTEPartition std::unique_ptr mu; Blocks blocks; std::unordered_map fetch_block_idxs; - size_t memory_usage = 0; - size_t memory_threoshold = 0; std::unique_ptr pipe_cv; - // Protecting cte_status and tmp_blocks - std::unique_ptr aux_lock; - CTEPartitionStatus status = CTEPartitionStatus::NORMAL; - Blocks tmp_blocks; - std::vector block_in_disk_nums; std::unordered_map spillers; std::unordered_map cte_reader_restore_streams; From 2a64cf368753059889f30a84fdd68d2f32c3611c Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 21 Jul 2025 11:08:31 +0800 Subject: [PATCH 102/118] fix ci --- .../Pipeline/Schedule/Events/tests/gtest_event.cpp | 2 +- .../Schedule/TaskQueues/tests/gtest_io_priority.cpp | 4 ++-- .../Schedule/TaskQueues/tests/gtest_mlfq.cpp | 4 ++-- .../tests/gtest_resource_control_queue.cpp | 12 ++---------- .../Pipeline/Schedule/tests/gtest_task_scheduler.cpp | 2 +- dbms/src/Operators/CTESourceOp.h | 2 +- 6 files changed, 9 insertions(+), 17 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp index 6f30bbf52d5..643153e2d30 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp @@ -519,7 +519,7 @@ CATCH TEST_F(EventTestRunner, memoryTrace) try { - PipelineExecutorContext exec_context{"", "", "", MemoryTracker::create()}; + PipelineExecutorContext exec_context{"", "", MemoryTracker::create()}; auto event = std::make_shared(exec_context); if (event->prepare()) event->schedule(); diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp index 45210f4b3b6..91a62086f4b 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp @@ -180,12 +180,12 @@ CATCH TEST_F(TestIOPriorityTaskQueue, cancel) try { - PipelineExecutorContext context1("id1", "id1", "", nullptr); + PipelineExecutorContext context1("id1", "", nullptr); // To avoid the active ref count being returned to 0 in advance. context1.incActiveRefCount(); SCOPE_EXIT({ context1.decActiveRefCount(); }); - PipelineExecutorContext context2("id2", "id2", "", nullptr); + PipelineExecutorContext context2("id2", "", nullptr); // To avoid the active ref count being returned to 0 in advance. context2.incActiveRefCount(); SCOPE_EXIT({ context2.decActiveRefCount(); }); diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp index d3a3b6bf241..450af49fa94 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp @@ -236,12 +236,12 @@ CATCH TEST_F(TestMLFQTaskQueue, cancel) try { - PipelineExecutorContext context1("id1", "id1", "", nullptr); + PipelineExecutorContext context1("id1", "", nullptr); // To avoid the active ref count being returned to 0 in advance. context1.incActiveRefCount(); SCOPE_EXIT({ context1.decActiveRefCount(); }); - PipelineExecutorContext context2("id2", "id2", "", nullptr); + PipelineExecutorContext context2("id2", "", nullptr); // To avoid the active ref count being returned to 0 in advance. context2.incActiveRefCount(); SCOPE_EXIT({ context2.decActiveRefCount(); }); diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp index a26212fde06..e8f102996f5 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp @@ -454,16 +454,8 @@ TEST_F(TestResourceControlQueue, RunOutOfRU) TaskSchedulerConfig config{thread_num, thread_num}; TaskScheduler task_scheduler(config); - PipelineExecutorContext exec_context( - "mock-query-id", - "mock-query-id-for-cte", - "mock-req-id", - mem_tracker, - nullptr, - nullptr, - nullptr, - NullspaceID, - rg_name); + PipelineExecutorContext + exec_context("mock-query-id", "mock-req-id", mem_tracker, nullptr, nullptr, nullptr, NullspaceID, rg_name); auto task = std::make_unique(exec_context); // This task should use 5*100ms cpu_time. diff --git a/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp b/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp index f613f2af88b..c69a84b3a30 100644 --- a/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp @@ -224,7 +224,7 @@ try { for (size_t task_num = 1; task_num < 100; ++task_num) { - PipelineExecutorContext exec_context{"", "", "", MemoryTracker::create()}; + PipelineExecutorContext exec_context{"", "", MemoryTracker::create()}; std::vector tasks; for (size_t i = 0; i < task_num; ++i) tasks.push_back(std::make_unique(exec_context)); diff --git a/dbms/src/Operators/CTESourceOp.h b/dbms/src/Operators/CTESourceOp.h index 1975fe88d41..ed5312c6a6c 100644 --- a/dbms/src/Operators/CTESourceOp.h +++ b/dbms/src/Operators/CTESourceOp.h @@ -15,13 +15,13 @@ #pragma once #include +#include #include #include #include #include #include #include -#include #include From 6b90cadee0159b85b787705ead0e84bc930d05dd Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 21 Jul 2025 17:26:56 +0800 Subject: [PATCH 103/118] fix --- .../Flash/Planner/Plans/PhysicalCTESink.cpp | 2 +- dbms/src/Interpreters/CTESpillContext.cpp | 27 ++---- dbms/src/Interpreters/CTESpillContext.h | 83 ++++-------------- dbms/src/Operators/CTE.cpp | 44 ++++++---- dbms/src/Operators/CTE.h | 20 +++-- dbms/src/Operators/CTEPartition.cpp | 44 +++++----- dbms/src/Operators/CTEPartition.h | 85 +++++++++++++++---- 7 files changed, 149 insertions(+), 156 deletions(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp index 9a007ecc610..a74feda990e 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalCTESink.cpp @@ -71,7 +71,7 @@ void PhysicalCTESink::buildPipelineExecGroupImpl( settings.max_threads, settings.max_block_size); - cte->initCTESpillContext( + cte->initCTESpillContextAndPartitionConfig( spill_config, group_builder.getCurrentHeader(), settings.max_bytes_before_cte_spill, diff --git a/dbms/src/Interpreters/CTESpillContext.cpp b/dbms/src/Interpreters/CTESpillContext.cpp index adbf3384b7a..ef6defefb44 100644 --- a/dbms/src/Interpreters/CTESpillContext.cpp +++ b/dbms/src/Interpreters/CTESpillContext.cpp @@ -19,31 +19,18 @@ namespace DB { -SpillerPtr CTESpillContext::getSpiller(size_t partition_id, size_t spill_id) -{ - SpillConfig config( - this->spill_config.spill_dir, - fmt::format("cte_spill_{}_{}", partition_id, spill_id), - this->spill_config.max_cached_data_bytes_in_spiller, - this->spill_config.max_spilled_rows_per_file, - this->spill_config.max_spilled_bytes_per_file, - this->spill_config.file_provider, - this->spill_config.for_all_constant_max_streams, - this->spill_config.for_all_constant_block_size); - - return std::make_unique(config, false, this->partition_num, this->spill_block_schema, this->log, 1, false); -} - Int64 CTESpillContext::triggerSpillImpl(Int64 expected_released_memories) { - for (size_t i = 0; i < this->partition_num; i++) + for (auto & partition : this->partitions) { - std::unique_lock lock(this->aux_locks[i]); - if (this->statuses[i] != CTEPartitionStatus::NORMAL) + std::lock_guard aux_lock(*(partition->aux_lock)); + if (partition->status != CTEPartitionStatus::NORMAL) continue; - this->statuses[i] = CTEPartitionStatus::NEED_SPILL; - expected_released_memories = std::max(expected_released_memories - this->memory_usages[i], 0); + partition->status = CTEPartitionStatus::NEED_SPILL; + + std::lock_guard lock(*(partition->mu)); + expected_released_memories = std::max(expected_released_memories - partition->memory_usage, 0); if (expected_released_memories <= 0) return expected_released_memories; } diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h index 6fdca7658f1..d280d74d49b 100644 --- a/dbms/src/Interpreters/CTESpillContext.h +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -15,96 +15,45 @@ #pragma once #include -#include #include #include -#include -#include +#include -#include +#include +#include + +#include "Common/Logger.h" namespace DB { -enum CTEPartitionStatus -{ - NORMAL = 0, - NEED_SPILL, - IN_SPILLING, -}; - class CTESpillContext final : public OperatorSpillContext { public: CTESpillContext( UInt64 operator_spill_threshold_, - size_t partition_num_, - const SpillConfig & spill_config_, - const Block & spill_block_schema_, - const String & query_id_and_cte_id_) + const String & query_id_and_cte_id_, + std::vector> partitions_) : OperatorSpillContext(operator_spill_threshold_, "cte", Logger::get(query_id_and_cte_id_)) - , partition_num(partition_num_) - , partition_memory_threoshold(operator_spill_threshold_ / partition_num_) - , memory_usages(partition_num_) - , spill_config(spill_config_) - , spill_block_schema(spill_block_schema_) - , query_id_and_cte_id(query_id_and_cte_id_) - , aux_locks(partition_num) - , statuses(partition_num) - , tmp_blocks(partition_num) - { - for (auto & status : this->statuses) - status = CTEPartitionStatus::NORMAL; - } - - SpillerPtr getSpiller(size_t partition_id, size_t spill_id); - LoggerPtr getLog() const { return this->log; } - String getQueryIdAndCTEId() const { return this->query_id_and_cte_id; } + , partitions(partitions_) + {} bool supportAutoTriggerSpill() const override { return true; } Int64 triggerSpillImpl(Int64 expected_released_memories) override; - - std::mutex * getPartitionAuxMutex(size_t partition_id) { return &(this->aux_locks[partition_id]); } - - CTEPartitionStatus getPartitionStatusNoLock(size_t partition_id) const { return this->statuses[partition_id]; } - void setPartitionStatusNoLock(size_t partition_id, CTEPartitionStatus status) - { - this->statuses[partition_id] = status; - } - - void pushTmpBlock(size_t partition_id, const Block & block) { this->tmp_blocks[partition_id].push_back(block); } - Blocks & getTmpBlocks(size_t partition_id) { return this->tmp_blocks[partition_id]; } - - void clearMemoryUsage(size_t partition_id) { this->memory_usages[partition_id].store(0); } - void addMemoryUsage(size_t partition_id, size_t delta) { this->memory_usages[partition_id].fetch_add(delta); } - bool exceedMemoryThreshold(size_t partition_id) const - { - if (this->partition_memory_threoshold == 0) - return false; - return this->memory_usages[partition_id].load() > this->partition_memory_threoshold; - } + LoggerPtr getLog() const { return this->log; } protected: Int64 getTotalRevocableMemoryImpl() override { Int64 total_memory = 0; - for (const auto & memory_usage : this->memory_usages) - total_memory += memory_usage.load(); + for (auto & partition : this->partitions) + { + std::lock_guard lock(*(partition->mu)); + total_memory += partition->memory_usage; + } return total_memory; } private: - size_t partition_num; - size_t partition_memory_threoshold; - std::vector memory_usages; - - SpillConfig spill_config; - Block spill_block_schema; - - String query_id_and_cte_id; - - // Protecting cte_status and tmp_blocks - std::vector aux_locks; - std::vector statuses; - std::vector tmp_blocks; + std::vector> partitions; }; } // namespace DB diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 52550f33a85..eca02671d6a 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -28,21 +28,31 @@ namespace DB { -void CTE::initCTESpillContext( +void CTE::initCTESpillContextAndPartitionConfig( const SpillConfig & spill_config, const Block & spill_block_schema, UInt64 operator_spill_threshold, Context & context) { std::unique_lock lock(this->rw_lock); - this->cte_spill_context = std::make_shared( - operator_spill_threshold, - this->partition_num, + if (this->cte_spill_context) + // Initialization has been executed before + return; + + const auto & query_id_and_cte_id = context.getDAGContext()->getQueryIDAndCTEIDForSink(); + this->cte_spill_context + = std::make_shared(operator_spill_threshold, query_id_and_cte_id, this->partitions); + + this->partition_config = std::make_shared( + operator_spill_threshold / this->partition_num, spill_config, spill_block_schema, - context.getDAGContext()->getQueryIDAndCTEIDForSink()); - for (auto & item : this->partitions) - item.init(this->cte_spill_context); + query_id_and_cte_id, + this->cte_spill_context->getLog(), + this->partition_num); + + for (auto & partition : this->partitions) + partition->setSharedConfig(this->partition_config); context.getDAGContext()->registerOperatorSpillContext(this->cte_spill_context); } @@ -56,7 +66,7 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block if unlikely (!this->areAllSinksRegistered()) return CTEOpStatus::SINK_NOT_REGISTERED; - auto status = this->partitions[partition_id].tryGetBlock(cte_reader_id, block); + auto status = this->partitions[partition_id]->tryGetBlock(cte_reader_id, block); switch (status) { case CTEOpStatus::BLOCK_NOT_AVAILABLE: @@ -75,7 +85,7 @@ CTEOpStatus CTE::pushBlock(size_t partition_id, const Block & block) if unlikely (this->is_cancelled) return CTEOpStatus::CANCELLED; - return this->partitions[partition_id].pushBlock(block); + return this->partitions[partition_id]->pushBlock(block); } CTEOpStatus CTE::getBlockFromDisk(size_t cte_reader_id, size_t partition_id, Block & block) @@ -86,7 +96,7 @@ CTEOpStatus CTE::getBlockFromDisk(size_t cte_reader_id, size_t partition_id, Blo return CTEOpStatus::CANCELLED; } - return this->partitions[partition_id].getBlockFromDisk(cte_reader_id, block); + return this->partitions[partition_id]->getBlockFromDisk(cte_reader_id, block); } CTEOpStatus CTE::spillBlocks(size_t partition_id) @@ -97,7 +107,7 @@ CTEOpStatus CTE::spillBlocks(size_t partition_id) return CTEOpStatus::CANCELLED; } - return this->partitions[partition_id].spillBlocks(); + return this->partitions[partition_id]->spillBlocks(); } void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t partition_id) @@ -109,15 +119,15 @@ void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_ return; } - std::lock_guard aux_lock(*(this->partitions[partition_id].getAuxMutex())); - if (this->partitions[partition_id].getStatusNoLock() == CTEPartitionStatus::IN_SPILLING) + std::lock_guard aux_lock(*(this->partitions[partition_id]->aux_lock)); + if (this->partitions[partition_id]->status == CTEPartitionStatus::IN_SPILLING) { this->notifyTaskDirectly(partition_id, std::move(task)); return; } - std::lock_guard lock(*(this->partitions[partition_id].mu)); - if (this->partitions[partition_id].isBlockAvailableNoLock(cte_reader_id) || this->is_eof) + std::lock_guard lock(*(this->partitions[partition_id]->mu)); + if (this->partitions[partition_id]->isBlockAvailableNoLock(cte_reader_id) || this->is_eof) this->notifyTaskDirectly(partition_id, std::move(task)); else this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); @@ -132,8 +142,8 @@ void CTE::checkInSpillingAndRegisterTask(TaskPtr && task, size_t partition_id) return; } - std::lock_guard aux_lock(*(this->partitions[partition_id].getAuxMutex())); - if (this->partitions[partition_id].getStatusNoLock() == CTEPartitionStatus::IN_SPILLING) + std::lock_guard aux_lock(*(this->partitions[partition_id]->aux_lock)); + if (this->partitions[partition_id]->status == CTEPartitionStatus::IN_SPILLING) this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE); else this->notifyTaskDirectly(partition_id, std::move(task)); diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index a44a1eec7aa..9c4c06e69d2 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include @@ -37,16 +38,16 @@ class CTE { RUNTIME_CHECK(this->partition_num > 0); for (size_t i = 0; i < this->partition_num; i++) - this->partitions.push_back(CTEPartition(i)); + this->partitions.push_back(std::make_shared(i)); } ~CTE() { for (auto & p : this->partitions) - p.debugOutput(); + p->debugOutput(); } - void initCTESpillContext( + void initCTESpillContextAndPartitionConfig( const SpillConfig & spill_config, const Block & spill_block_schema, UInt64 operator_spill_threshold, @@ -67,7 +68,7 @@ class CTE auto cte_reader_id = this->next_cte_reader_id; this->next_cte_reader_id++; for (auto & item : this->partitions) - item.fetch_block_idxs.insert(std::make_pair(cte_reader_id, 0)); + item->fetch_block_idxs.insert(std::make_pair(cte_reader_id, 0)); return cte_reader_id; } @@ -91,12 +92,12 @@ class CTE void registerTask(size_t partition_id, TaskPtr && task, NotifyType type) { task->setNotifyType(type); - this->partitions[partition_id].pipe_cv->registerTask(std::move(task)); + this->partitions[partition_id]->pipe_cv->registerTask(std::move(task)); } void notifyTaskDirectly(size_t partition_id, TaskPtr && task) { - this->partitions[partition_id].pipe_cv->notifyTaskDirectly(std::move(task)); + this->partitions[partition_id]->pipe_cv->notifyTaskDirectly(std::move(task)); } void addResp(const tipb::SelectResponse & resp) @@ -130,7 +131,7 @@ class CTE return this->registered_sink_num == this->expected_sink_num; } - LoggerPtr getLog() const { return this->cte_spill_context->getLog(); } + LoggerPtr getLog() const { return this->partition_config->log; } private: void notifyImpl(bool is_eof, const String & msg) @@ -146,13 +147,13 @@ class CTE } for (auto & partition : this->partitions) - partition.pipe_cv->notifyAll(); + partition->pipe_cv->notifyAll(); } size_t next_cte_reader_id = 0; size_t partition_num; - std::vector partitions; + std::vector> partitions; std::shared_mutex rw_lock; bool is_eof = false; @@ -166,6 +167,7 @@ class CTE String err_msg; std::shared_ptr cte_spill_context; + std::shared_ptr partition_config; }; class CTEIONotifier : public NotifyFuture diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 330e2916039..191ba35425f 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -29,8 +29,8 @@ size_t CTEPartition::getIdxInMemoryNoLock(size_t cte_reader_id) CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) { - std::lock_guard aux_lock(*(this->getAuxMutex())); - if (this->getStatusNoLock() == CTEPartitionStatus::IN_SPILLING) + std::lock_guard aux_lock(*(this->aux_lock)); + if (this->status == CTEPartitionStatus::IN_SPILLING) return CTEOpStatus::WAIT_SPILL; std::lock_guard lock(*this->mu); @@ -57,9 +57,9 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) CTEOpStatus CTEPartition::pushBlock(const Block & block) { - std::unique_lock aux_lock(*(this->getAuxMutex())); + std::unique_lock aux_lock(*(this->aux_lock)); CTEOpStatus ret_status = CTEOpStatus::OK; - switch (this->getStatusNoLock()) + switch (this->status) { case CTEPartitionStatus::NEED_SPILL: ret_status = CTEOpStatus::NEED_SPILL; @@ -67,7 +67,7 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) ret_status = CTEOpStatus::WAIT_SPILL; if likely (block.rows() != 0) // Block memory usage will be calculated after the finish of spill - this->pushTmpBlock(block); + this->tmp_blocks.push_back(block); return ret_status; case CTEPartitionStatus::NORMAL: break; @@ -81,13 +81,13 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) this->total_recv_row_num += block.rows(); this->total_byte_usage += block.bytes(); - this->addMemoryUsage(block.bytes()); + this->memory_usage += block.bytes(); this->blocks.push_back(block); this->pipe_cv->notifyOne(); if unlikely (this->exceedMemoryThresholdNoLock()) { - this->setCTEPartitionStatusNoLock(CTEPartitionStatus::NEED_SPILL); + this->status = CTEPartitionStatus::NEED_SPILL; ret_status = CTEOpStatus::NEED_SPILL; } return ret_status; @@ -96,30 +96,26 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) CTEOpStatus CTEPartition::spillBlocks() { LOG_INFO( - this->spill_context->getLog(), - fmt::format( - "Partition {} starts cte spill for {}", - this->partition_id, - this->spill_context->getQueryIdAndCTEId())); + this->config->log, + fmt::format("Partition {} starts cte spill for {}", this->partition_id, this->config->query_id_and_cte_id)); std::unique_lock lock(*(this->mu), std::defer_lock); { - std::lock_guard aux_lock(*(this->getAuxMutex())); - switch (this->getStatusNoLock()) + std::lock_guard aux_lock(*(this->aux_lock)); + switch (this->status) { case CTEPartitionStatus::NORMAL: return CTEOpStatus::OK; case CTEPartitionStatus::IN_SPILLING: return CTEOpStatus::WAIT_SPILL; case CTEPartitionStatus::NEED_SPILL: - this->setCTEPartitionStatusNoLock(CTEPartitionStatus::IN_SPILLING); + this->status = CTEPartitionStatus::IN_SPILLING; break; } lock.lock(); - Blocks & tmp_blocks = this->getTmpBlocks(); - for (const auto & block : tmp_blocks) + for (const auto & block : this->tmp_blocks) { - this->addMemoryUsage(block.bytes()); + this->memory_usage += block.bytes(); this->blocks.push_back(block); } tmp_blocks.clear(); @@ -155,7 +151,7 @@ CTEOpStatus CTEPartition::spillBlocks() this->total_spill_block_num += spilled_blocks.size(); // TODO remove - auto spiller = this->spill_context->getSpiller(this->partition_id, this->spillers.size()); + auto spiller = this->config->getSpiller(this->partition_id, this->spillers.size()); spiller->spillBlocks(std::move(spilled_blocks), this->partition_id); spiller->finishSpill(); this->spillers.insert(std::make_pair(split_iter->first, std::move(spiller))); @@ -163,10 +159,10 @@ CTEOpStatus CTEPartition::spillBlocks() } this->blocks.clear(); - this->clearMemoryUsage(); + this->memory_usage = 0; - std::lock_guard aux_lock(*(this->getAuxMutex())); - this->setCTEPartitionStatusNoLock(CTEPartitionStatus::NORMAL); + std::lock_guard aux_lock(*(this->aux_lock)); + this->status = CTEPartitionStatus::NORMAL; // Many tasks may be waiting for the finish of spill this->pipe_cv->notifyAll(); @@ -177,8 +173,8 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) { std::unique_lock lock(*(this->mu), std::defer_lock); { - std::lock_guard aux_lock(*(this->getAuxMutex())); - if (this->getStatusNoLock() == CTEPartitionStatus::IN_SPILLING) + std::lock_guard aux_lock(*(this->aux_lock)); + if (this->status == CTEPartitionStatus::IN_SPILLING) return CTEOpStatus::WAIT_SPILL; lock.lock(); diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 14c3b16beb5..905872dd4ac 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -14,9 +14,9 @@ #pragma once +#include #include #include -#include #include #include @@ -25,6 +25,13 @@ namespace DB { +enum CTEPartitionStatus +{ + NORMAL = 0, + NEED_SPILL, + IN_SPILLING, +}; + enum class CTEOpStatus { OK, @@ -37,10 +44,59 @@ enum class CTEOpStatus SINK_NOT_REGISTERED }; +struct CTEPartitionSharedConfig +{ + CTEPartitionSharedConfig( + size_t memory_threshold_, + SpillConfig spill_config_, + Block spill_block_schema_, + String query_id_and_cte_id_, + LoggerPtr log_, + size_t partition_num_) + : memory_threshold(memory_threshold_) + , spill_config(spill_config_) + , spill_block_schema(spill_block_schema_) + , query_id_and_cte_id(query_id_and_cte_id_) + , log(log_) + , partition_num(partition_num_) + {} + + SpillerPtr getSpiller(size_t partition_id, size_t spill_id) + { + SpillConfig config( + this->spill_config.spill_dir, + fmt::format("cte_spill_{}_{}", partition_id, spill_id), + this->spill_config.max_cached_data_bytes_in_spiller, + this->spill_config.max_spilled_rows_per_file, + this->spill_config.max_spilled_bytes_per_file, + this->spill_config.file_provider, + this->spill_config.for_all_constant_max_streams, + this->spill_config.for_all_constant_block_size); + + return std::make_unique( + config, + false, + this->partition_num, + this->spill_block_schema, + this->log, + 1, + false); + } + + size_t memory_threshold; + SpillConfig spill_config; + Block spill_block_schema; + String query_id_and_cte_id; + LoggerPtr log; + size_t partition_num; +}; + struct CTEPartition { explicit CTEPartition(size_t partition_id_) : partition_id(partition_id_) + , aux_lock(std::make_unique()) + , status(CTEPartitionStatus::NORMAL) , mu(std::make_unique()) , pipe_cv(std::make_unique()) {} @@ -91,7 +147,7 @@ struct CTEPartition infos)); } - void init(std::shared_ptr spill_context_) { this->spill_context = spill_context_; } + void setSharedConfig(std::shared_ptr config) { this->config = config; } size_t getIdxInMemoryNoLock(size_t cte_reader_id); bool isBlockAvailableInDiskNoLock(size_t cte_reader_id) @@ -102,13 +158,10 @@ struct CTEPartition { return this->getIdxInMemoryNoLock(cte_reader_id) < this->blocks.size(); } - void setCTEPartitionStatusNoLock(CTEPartitionStatus status) const - { - this->spill_context->setPartitionStatusNoLock(this->partition_id, status); - } + bool isSpillTriggeredNoLock() const { return this->total_block_in_disk_num > 0; } void addIdxNoLock(size_t cte_reader_id) { this->fetch_block_idxs[cte_reader_id]++; } - bool exceedMemoryThresholdNoLock() const { return this->spill_context->exceedMemoryThreshold(this->partition_id); } + bool exceedMemoryThresholdNoLock() const { return this->memory_usage >= this->config->memory_threshold; } CTEOpStatus pushBlock(const Block & block); CTEOpStatus tryGetBlock(size_t cte_reader_id, Block & block); @@ -123,16 +176,6 @@ struct CTEPartition return this->isBlockAvailableInMemoryNoLock(cte_reader_id); } - std::mutex * getAuxMutex() const { return this->spill_context->getPartitionAuxMutex(this->partition_id); } - CTEPartitionStatus getStatusNoLock() const - { - return this->spill_context->getPartitionStatusNoLock(this->partition_id); - } - void pushTmpBlock(const Block & block) const { this->spill_context->pushTmpBlock(this->partition_id, block); } - Blocks & getTmpBlocks() const { return this->spill_context->getTmpBlocks(this->partition_id); } - void addMemoryUsage(size_t delta) const { this->spill_context->addMemoryUsage(this->partition_id, delta); } - void clearMemoryUsage() const { this->spill_context->clearMemoryUsage(this->partition_id); } - size_t total_recv_block_num = 0; size_t total_recv_row_num = 0; size_t total_spill_block_num = 0; @@ -145,16 +188,22 @@ struct CTEPartition size_t partition_id; + std::unique_ptr aux_lock; + CTEPartitionStatus status; + Blocks tmp_blocks; + std::unique_ptr mu; Blocks blocks; std::unordered_map fetch_block_idxs; std::unique_ptr pipe_cv; + size_t memory_usage = 0; + std::vector block_in_disk_nums; std::unordered_map spillers; std::unordered_map cte_reader_restore_streams; UInt64 total_block_in_disk_num = 0; - std::shared_ptr spill_context; + std::shared_ptr config; }; } // namespace DB From 4660656c6db5b8a12d8fcfc822a3cbcb727e69b9 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 22 Jul 2025 09:05:47 +0800 Subject: [PATCH 104/118] fix ci --- .../Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp index e8f102996f5..0f6fbff0f05 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp @@ -179,7 +179,6 @@ class TestResourceControlQueue : public ::testing::Test { auto resource_group_name = resource_groups[i]->name; all_contexts[i] = std::make_shared( - query_id_prefix + resource_group_name, query_id_prefix + resource_group_name, req_id_prefix + resource_group_name, mem_tracker, @@ -380,7 +379,6 @@ TEST_F(TestResourceControlQueue, BasicTest) { String group_name = "rg" + std::to_string(i); all_contexts[i] = std::make_shared( - "mock-query-id", "mock-query-id", "mock-req-id", mem_tracker, @@ -418,7 +416,6 @@ TEST_F(TestResourceControlQueue, BasicTimeoutTest) // In the destructor of Task, will use PipelineExecutorContext to log. String group_name = "rg1"; auto exec_context = std::make_shared( - "mock-query-id", "mock-query-id", "mock-req-id", mem_tracker, From 0384d2d9a0174bc1711ec94224748b8bb6024b0d Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 22 Jul 2025 13:06:31 +0800 Subject: [PATCH 105/118] add logs --- dbms/src/Flash/Mpp/CTEManager.cpp | 26 ++------- dbms/src/Flash/Mpp/MPPTask.cpp | 8 +++ dbms/src/Interpreters/CTESpillContext.cpp | 8 +++ dbms/src/Operators/CTEPartition.h | 65 ++++++++++++----------- 4 files changed, 53 insertions(+), 54 deletions(-) diff --git a/dbms/src/Flash/Mpp/CTEManager.cpp b/dbms/src/Flash/Mpp/CTEManager.cpp index 224feba806b..bc845f987f6 100644 --- a/dbms/src/Flash/Mpp/CTEManager.cpp +++ b/dbms/src/Flash/Mpp/CTEManager.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -29,20 +30,9 @@ void CTEManager::releaseCTEBySource(const String & query_id_and_cte_id) // Maybe the task is cancelled and the cte has been released return; - auto * log = &Poco::Logger::get("LRUCache"); - iter->second.sourceExit(); - LOG_INFO( - log, - fmt::format( - "xzxdebug total exit: {}, expect: {}", - iter->second.getTotalExitNum(), - iter->second.getExpectedTotalNum())); if (iter->second.getTotalExitNum() == iter->second.getExpectedTotalNum()) - { - LOG_INFO(log, "xzxdebug erase"); this->ctes.erase(iter); - } } void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const String & query_id_and_cte_id) @@ -52,26 +42,14 @@ void CTEManager::releaseCTEBySink(const tipb::SelectResponse & resp, const Strin if unlikely (iter == this->ctes.end()) // Maybe the task is cancelled and the cte has been released return; - auto * log = &Poco::Logger::get("LRUCache"); CTEWithCounter & cte_with_counter = iter->second; cte_with_counter.getCTE()->addResp(resp); cte_with_counter.sinkExit(); - LOG_INFO( - log, - fmt::format( - "xzxdebug total sink: {}, expect sink: {}, total exit: {}, expect: {}", - cte_with_counter.getSinkExitNum(), - cte_with_counter.getExpectedSinkNum(), - cte_with_counter.getTotalExitNum(), - cte_with_counter.getExpectedTotalNum())); if (cte_with_counter.getSinkExitNum() == cte_with_counter.getExpectedSinkNum()) cte_with_counter.getCTE()->notifyEOF(); if (cte_with_counter.getTotalExitNum() == cte_with_counter.getExpectedTotalNum()) - { - LOG_INFO(log, "xzxdebug erase"); this->ctes.erase(iter); - } } void CTEManager::releaseCTE(const String & query_id_and_cte_id) @@ -88,6 +66,8 @@ std::shared_ptr CTEManager::getCTE( Int32 expected_sink_num, Int32 expected_source_num) { + RUNTIME_CHECK(concurrency > 0); + std::lock_guard lock(this->mu); auto iter = this->ctes.find(query_id_and_cte_id); if (iter == this->ctes.end()) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 6a617706a44..e1d9be8ec47 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -613,6 +613,14 @@ void MPPTask::runImpl() scheduleOrWait(); + auto sink = dag_context->getCTESink(); + auto source = dag_context->getCTESource(); + if (sink && source) + { + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, "xzxdebug both sink and source are set in the same mpp task"); + } + auto time_cost_in_schedule_ns = stopwatch.elapsed() - time_cost_in_preprocess_ns; dag_context->minTSO_wait_time_ns = time_cost_in_schedule_ns; auto time_cost_in_schedule_ms = time_cost_in_schedule_ns / MILLISECOND_TO_NANO; diff --git a/dbms/src/Interpreters/CTESpillContext.cpp b/dbms/src/Interpreters/CTESpillContext.cpp index ef6defefb44..bd24af88f9a 100644 --- a/dbms/src/Interpreters/CTESpillContext.cpp +++ b/dbms/src/Interpreters/CTESpillContext.cpp @@ -21,6 +21,9 @@ namespace DB { Int64 CTESpillContext::triggerSpillImpl(Int64 expected_released_memories) { + String info = "xzxdebug CTE auto spill is triggered, spilled partitions: "; + auto * log = &Poco::Logger::get("LRUCache"); + for (auto & partition : this->partitions) { std::lock_guard aux_lock(*(partition->aux_lock)); @@ -28,12 +31,17 @@ Int64 CTESpillContext::triggerSpillImpl(Int64 expected_released_memories) continue; partition->status = CTEPartitionStatus::NEED_SPILL; + info = fmt::format("{}, {}", info, partition->partition_id); std::lock_guard lock(*(partition->mu)); expected_released_memories = std::max(expected_released_memories - partition->memory_usage, 0); if (expected_released_memories <= 0) + { + LOG_INFO(log, info); return expected_released_memories; + } } + LOG_INFO(log, info); return expected_released_memories; } } // namespace DB diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 905872dd4ac..896753c6abe 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -59,7 +59,10 @@ struct CTEPartitionSharedConfig , query_id_and_cte_id(query_id_and_cte_id_) , log(log_) , partition_num(partition_num_) - {} + { + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, fmt::format("xzxdebug memory threshold is set for {}", this->memory_threshold)); + } SpillerPtr getSpiller(size_t partition_id, size_t spill_id) { @@ -103,23 +106,23 @@ struct CTEPartition void debugOutput() { - String info_block; - for (const auto & item : this->total_fetch_block_nums) - info_block = fmt::format("{} <{}: {}>", info_block, item.first, item.second); + // String info_block; + // for (const auto & item : this->total_fetch_block_nums) + // info_block = fmt::format("{} <{}: {}>", info_block, item.first, item.second); - String info_row; - for (const auto & item : this->total_fetch_row_nums) - info_row = fmt::format("{} <{}: {}>", info_row, item.first, item.second); + // String info_row; + // for (const auto & item : this->total_fetch_row_nums) + // info_row = fmt::format("{} <{}: {}>", info_row, item.first, item.second); - String disk_info_block; - for (const auto & item : this->total_fetch_disk_block_nums) - disk_info_block = fmt::format("{} <{}: {}>", disk_info_block, item.first, item.second); + // String disk_info_block; + // for (const auto & item : this->total_fetch_disk_block_nums) + // disk_info_block = fmt::format("{} <{}: {}>", disk_info_block, item.first, item.second); - String disk_info_row; - for (const auto & item : this->total_fetch_disk_row_nums) - disk_info_row = fmt::format("{} <{}: {}>", disk_info_row, item.first, item.second); + // String disk_info_row; + // for (const auto & item : this->total_fetch_disk_row_nums) + // disk_info_row = fmt::format("{} <{}: {}>", disk_info_row, item.first, item.second); - String infos; + // String infos; // for (const auto & item : this->fetch_idxs_disk) // { // String nums; @@ -128,23 +131,23 @@ struct CTEPartition // infos = fmt::format("{} ", infos, item.first, nums); // } - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO( - log, - fmt::format( - "xzxdebug CTEPartition total_recv_block_num: {}, row: {}, total_spill_block_num: {}, " - "total_fetch_block_num: {}, row num: {}, " - "disk: {}, {}" - "total_byte_usage: {}, idxs_disk: {}", - total_recv_block_num, - total_recv_row_num, - total_spill_block_num, - info_block, - info_row, - disk_info_block, - disk_info_row, - total_byte_usage, - infos)); + // auto * log = &Poco::Logger::get("LRUCache"); + // LOG_INFO( + // log, + // fmt::format( + // "xzxdebug CTEPartition total_recv_block_num: {}, row: {}, total_spill_block_num: {}, " + // "total_fetch_block_num: {}, row num: {}, " + // "disk: {}, {}" + // "total_byte_usage: {}, idxs_disk: {}", + // total_recv_block_num, + // total_recv_row_num, + // total_spill_block_num, + // info_block, + // info_row, + // disk_info_block, + // disk_info_row, + // total_byte_usage, + // infos)); } void setSharedConfig(std::shared_ptr config) { this->config = config; } From 834b20d687e243edb061974b4a0d1620be9639c7 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 23 Jul 2025 11:19:15 +0800 Subject: [PATCH 106/118] fix block missing --- dbms/src/Operators/CTE.cpp | 33 ++++++++-- dbms/src/Operators/CTE.h | 37 ++++++++++++ dbms/src/Operators/CTEPartition.cpp | 51 +++++++--------- dbms/src/Operators/CTEPartition.h | 94 ++++++++++++++--------------- dbms/src/Operators/CTEReader.cpp | 3 + dbms/src/Operators/CTEReader.h | 11 ++++ 6 files changed, 146 insertions(+), 83 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index eca02671d6a..83a777aaf35 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -14,17 +14,14 @@ #include #include -#include -#include -#include #include #include -#include #include #include #include #include +#include namespace DB { @@ -69,6 +66,16 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block auto status = this->partitions[partition_id]->tryGetBlock(cte_reader_id, block); switch (status) { + case CTEOpStatus::OK: + { + auto [iter, _] = this->total_fetch_blocks.insert(std::make_pair(cte_reader_id, 0)); + iter->second.fetch_add(1); + } + { + auto [iter, _] = this->total_fetch_rows.insert(std::make_pair(cte_reader_id, 0)); + iter->second.fetch_add(block.rows()); + } + return status; case CTEOpStatus::BLOCK_NOT_AVAILABLE: return this->is_eof ? CTEOpStatus::END_OF_FILE : CTEOpStatus::BLOCK_NOT_AVAILABLE; default: @@ -85,6 +92,8 @@ CTEOpStatus CTE::pushBlock(size_t partition_id, const Block & block) if unlikely (this->is_cancelled) return CTEOpStatus::CANCELLED; + this->total_recv_blocks.fetch_add(1); + this->total_recv_rows.fetch_add(block.rows()); return this->partitions[partition_id]->pushBlock(block); } @@ -96,7 +105,19 @@ CTEOpStatus CTE::getBlockFromDisk(size_t cte_reader_id, size_t partition_id, Blo return CTEOpStatus::CANCELLED; } - return this->partitions[partition_id]->getBlockFromDisk(cte_reader_id, block); + auto ret = this->partitions[partition_id]->getBlockFromDisk(cte_reader_id, block); + if (ret == CTEOpStatus::OK && block) + { + { + auto [iter, _] = this->total_fetch_blocks_in_disk.insert(std::make_pair(cte_reader_id, 0)); + iter->second.fetch_add(1); + } + { + auto [iter, _] = this->total_fetch_rows_in_disk.insert(std::make_pair(cte_reader_id, 0)); + iter->second.fetch_add(block.rows()); + } + } + return ret; } CTEOpStatus CTE::spillBlocks(size_t partition_id) @@ -107,7 +128,7 @@ CTEOpStatus CTE::spillBlocks(size_t partition_id) return CTEOpStatus::CANCELLED; } - return this->partitions[partition_id]->spillBlocks(); + return this->partitions[partition_id]->spillBlocks(this->total_spilled_blocks, this->total_spilled_rows); } void CTE::checkBlockAvailableAndRegisterTask(TaskPtr && task, size_t cte_reader_id, size_t partition_id) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 9c4c06e69d2..9b5c2ee49e7 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -23,12 +23,22 @@ #include #include +#include #include #include #include namespace DB { + +inline String genInfo(const String & name, const std::map & data) +{ + String info = fmt::format("{}: ", name); + for (const auto & item : data) + info = fmt::format("{}, <{}, {}>", info, item.first, item.second.load()); + return info; +} + class CTE { public: @@ -41,8 +51,35 @@ class CTE this->partitions.push_back(std::make_shared(i)); } + std::atomic_size_t total_recv_blocks = 0; + std::atomic_size_t total_recv_rows = 0; + + std::atomic_size_t total_spilled_blocks = 0; + std::atomic_size_t total_spilled_rows = 0; + + std::map total_fetch_blocks; + std::map total_fetch_rows; + + std::map total_fetch_blocks_in_disk; + std::map total_fetch_rows_in_disk; + ~CTE() { + String info; + info = fmt::format( + "total_recv_blocks: {}, total_recv_rows: {}, total_spilled_blocks: {}, total_spilled_rows: {}, ", + total_recv_blocks.load(), + total_recv_rows.load(), + total_spilled_blocks.load(), + total_spilled_rows.load()); + info = fmt::format("{} | {}", info, genInfo("total_fetch_blocks", this->total_fetch_blocks)); + info = fmt::format("{} | {}", info, genInfo("total_fetch_rows", this->total_fetch_rows)); + info = fmt::format("{} | {}", info, genInfo("total_fetch_blocks_in_disk", this->total_fetch_blocks_in_disk)); + info = fmt::format("{} | {}", info, genInfo("total_fetch_rows_in_disk", this->total_fetch_rows_in_disk)); + + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO(log, fmt::format("xzxdebug CTE {}", info)); + for (auto & p : this->partitions) p->debugOutput(); } diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 191ba35425f..e1442842021 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -15,6 +15,7 @@ #include #include +#include #include #include #include @@ -35,6 +36,8 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) std::lock_guard lock(*this->mu); + this->putTmpBlocksIntoBlocksNoLock(); + if (this->isBlockAvailableInDiskNoLock(cte_reader_id)) return CTEOpStatus::IO_IN; @@ -43,21 +46,18 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) auto idx = this->getIdxInMemoryNoLock(cte_reader_id); block = this->blocks[idx]; - this->addIdxNoLock(cte_reader_id); { - auto [iter, _] = this->total_fetch_block_nums.insert(std::make_pair(cte_reader_id, 0)); - iter->second++; - } - { - auto [iter, _] = this->total_fetch_row_nums.insert(std::make_pair(cte_reader_id, 0)); - iter->second += block.rows(); + auto [iter, _] = this->fetch_in_mem_idxs.insert(std::make_pair(cte_reader_id, 0)); + iter->second.push_back(this->fetch_block_idxs[cte_reader_id]); } + this->addIdxNoLock(cte_reader_id); return CTEOpStatus::OK; } CTEOpStatus CTEPartition::pushBlock(const Block & block) { std::unique_lock aux_lock(*(this->aux_lock)); + this->total_blocks.fetch_add(1); CTEOpStatus ret_status = CTEOpStatus::OK; switch (this->status) { @@ -77,10 +77,6 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) // Blocked in cpu pool is very bad. std::lock_guard lock(*this->mu); - this->total_recv_block_num++; - this->total_recv_row_num += block.rows(); - this->total_byte_usage += block.bytes(); - this->memory_usage += block.bytes(); this->blocks.push_back(block); this->pipe_cv->notifyOne(); @@ -93,7 +89,7 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) return ret_status; } -CTEOpStatus CTEPartition::spillBlocks() +CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomic_size_t & row_num) { LOG_INFO( this->config->log, @@ -113,12 +109,7 @@ CTEOpStatus CTEPartition::spillBlocks() } lock.lock(); - for (const auto & block : this->tmp_blocks) - { - this->memory_usage += block.bytes(); - this->blocks.push_back(block); - } - tmp_blocks.clear(); + this->putTmpBlocksIntoBlocksNoLock(); } // Key represents logical index @@ -141,17 +132,26 @@ CTEOpStatus CTEPartition::spillBlocks() Blocks spilled_blocks; if (next_iter == split_idxs.end() || next_iter->second >= total_block_in_memory_num) + { + this->spill_ranges.push_back( + std::make_pair(split_iter->first, this->blocks.size() - split_iter->second + split_iter->first)); spilled_blocks.assign(blocks_begin_iter + split_iter->second, this->blocks.end()); + } else + { + this->spill_ranges.push_back(std::make_pair(split_iter->first, next_iter->first)); spilled_blocks.assign(blocks_begin_iter + split_iter->second, blocks_begin_iter + next_iter->second); + } RUNTIME_CHECK(!spilled_blocks.empty()); this->total_block_in_disk_num += spilled_blocks.size(); - this->total_spill_block_num += spilled_blocks.size(); // TODO remove - auto spiller = this->config->getSpiller(this->partition_id, this->spillers.size()); + this->total_spill_blocks.fetch_add(spilled_blocks.size()); + block_num.fetch_add(spilled_blocks.size()); + for (auto & block : spilled_blocks) + row_num.fetch_add(block.rows()); spiller->spillBlocks(std::move(spilled_blocks), this->partition_id); spiller->finishSpill(); this->spillers.insert(std::make_pair(split_iter->first, std::move(spiller))); @@ -210,19 +210,12 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) retry = true; continue; } + { - auto [iter, _] = this->fetch_idxs_disk.insert(std::make_pair(cte_reader_id, std::vector{})); + auto [iter, _] = this->fetch_in_disk_idxs.insert(std::make_pair(cte_reader_id, 0)); iter->second.push_back(this->fetch_block_idxs[cte_reader_id]); } this->addIdxNoLock(cte_reader_id); - { - auto [iter, _] = this->total_fetch_disk_block_nums.insert(std::make_pair(cte_reader_id, 0)); - iter->second++; - } - { - auto [iter, _] = this->total_fetch_disk_row_nums.insert(std::make_pair(cte_reader_id, 0)); - iter->second += block.rows(); - } break; }; diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 896753c6abe..def34a3655d 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -18,6 +18,7 @@ #include #include +#include #include #include #include @@ -104,50 +105,32 @@ struct CTEPartition , pipe_cv(std::make_unique()) {} + // TODO remove it void debugOutput() { - // String info_block; - // for (const auto & item : this->total_fetch_block_nums) - // info_block = fmt::format("{} <{}: {}>", info_block, item.first, item.second); - - // String info_row; - // for (const auto & item : this->total_fetch_row_nums) - // info_row = fmt::format("{} <{}: {}>", info_row, item.first, item.second); - - // String disk_info_block; - // for (const auto & item : this->total_fetch_disk_block_nums) - // disk_info_block = fmt::format("{} <{}: {}>", disk_info_block, item.first, item.second); - - // String disk_info_row; - // for (const auto & item : this->total_fetch_disk_row_nums) - // disk_info_row = fmt::format("{} <{}: {}>", disk_info_row, item.first, item.second); - - // String infos; - // for (const auto & item : this->fetch_idxs_disk) + // String total_info = fmt::format("total_blocks: {}, total_spill_blocks: {}", total_blocks, total_spill_blocks); + // String spill_ranges_info = "spill_ranges: "; + // for (auto & range : this->spill_ranges) + // spill_ranges_info = fmt::format("{}, ({} - {})", spill_ranges_info, range.first, range.second); + // String fetch_in_mem_idxs_info = "fetch_in_mem_idxs: "; + // for (const auto & v : this->fetch_in_mem_idxs) // { - // String nums; - // for (auto idx : item.second) - // nums = fmt::format("{} {}", nums, idx); - // infos = fmt::format("{} ", infos, item.first, nums); + // String tmp_info; + // for (auto idx : v.second) + // tmp_info = fmt::format("{}, {}", tmp_info, idx); + // fetch_in_mem_idxs_info = fmt::format("{} <{}: {}>", fetch_in_mem_idxs_info, v.first, tmp_info); + // } + // String fetch_in_disk_idxs_info = "fetch_in_disk_idxs: "; + // for (const auto & v : this->fetch_in_disk_idxs) + // { + // String tmp_info; + // for (auto idx : v.second) + // tmp_info = fmt::format("{}, {}", tmp_info, idx); + // fetch_in_disk_idxs_info = fmt::format("{} <{}: {}>", fetch_in_disk_idxs_info, v.first, tmp_info); // } // auto * log = &Poco::Logger::get("LRUCache"); - // LOG_INFO( - // log, - // fmt::format( - // "xzxdebug CTEPartition total_recv_block_num: {}, row: {}, total_spill_block_num: {}, " - // "total_fetch_block_num: {}, row num: {}, " - // "disk: {}, {}" - // "total_byte_usage: {}, idxs_disk: {}", - // total_recv_block_num, - // total_recv_row_num, - // total_spill_block_num, - // info_block, - // info_row, - // disk_info_block, - // disk_info_row, - // total_byte_usage, - // infos)); + // LOG_INFO(log, fmt::format("xzxdebug | {} | {} | {} | {}", total_info, spill_ranges_info, fetch_in_mem_idxs_info, fetch_in_disk_idxs_info)); } void setSharedConfig(std::shared_ptr config) { this->config = config; } @@ -164,11 +147,16 @@ struct CTEPartition bool isSpillTriggeredNoLock() const { return this->total_block_in_disk_num > 0; } void addIdxNoLock(size_t cte_reader_id) { this->fetch_block_idxs[cte_reader_id]++; } - bool exceedMemoryThresholdNoLock() const { return this->memory_usage >= this->config->memory_threshold; } + bool exceedMemoryThresholdNoLock() const + { + if (this->config->memory_threshold == 0) + return false; + return this->memory_usage >= this->config->memory_threshold; + } CTEOpStatus pushBlock(const Block & block); CTEOpStatus tryGetBlock(size_t cte_reader_id, Block & block); - CTEOpStatus spillBlocks(); + CTEOpStatus spillBlocks(std::atomic_size_t & block_num, std::atomic_size_t & row_num); CTEOpStatus getBlockFromDisk(size_t cte_reader_id, Block & block); bool isBlockAvailableNoLock(size_t cte_reader_id) @@ -179,14 +167,24 @@ struct CTEPartition return this->isBlockAvailableInMemoryNoLock(cte_reader_id); } - size_t total_recv_block_num = 0; - size_t total_recv_row_num = 0; - size_t total_spill_block_num = 0; - std::map total_fetch_block_nums; - std::map total_fetch_row_nums; - std::map total_fetch_disk_block_nums; - std::map total_fetch_disk_row_nums; - std::map> fetch_idxs_disk; + // Need aux_lock and mu + void putTmpBlocksIntoBlocksNoLock() + { + for (const auto & block : this->tmp_blocks) + { + this->memory_usage += block.bytes(); + this->blocks.push_back(block); + } + tmp_blocks.clear(); + } + + std::atomic_size_t total_blocks = 0; + std::atomic_size_t total_spill_blocks = 0; + + std::vector> spill_ranges; + std::map> fetch_in_mem_idxs; + std::map> fetch_in_disk_idxs; + size_t total_byte_usage = 0; size_t partition_id; diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 2b7f8b9c578..e7396b327de 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -36,7 +36,10 @@ CTEOpStatus CTEReader::fetchNextBlock(size_t partition_id, Block & block) case CTEOpStatus::IO_IN: case CTEOpStatus::SINK_NOT_REGISTERED: case CTEOpStatus::BLOCK_NOT_AVAILABLE: + return ret; case CTEOpStatus::OK: + this->total_fetch_blocks.fetch_add(1); + this->total_fetch_rows.fetch_add(block.rows()); return ret; case CTEOpStatus::CANCELLED: throw Exception(this->cte->getError()); diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 6e7e53ef5ad..24f355b39e5 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -40,10 +40,21 @@ class CTEReader RUNTIME_CHECK(cte); } + std::atomic_size_t total_fetch_blocks = 0; + std::atomic_size_t total_fetch_rows = 0; + ~CTEReader() { this->cte.reset(); this->cte_manager->releaseCTEBySource(this->query_id_and_cte_id); + + auto * log = &Poco::Logger::get("LRUCache"); + LOG_INFO( + log, + fmt::format( + "xzxdebug CTEReader fb: {} fr: {}", + this->total_fetch_blocks.load(), + this->total_fetch_rows.load())); } CTEOpStatus fetchNextBlock(size_t partition_id, Block & block); From 3a058e8ffef1b073be431441a785fda3fcd567a7 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 29 Jul 2025 13:21:48 +0800 Subject: [PATCH 107/118] remove debug logs --- dbms/src/Flash/Mpp/MPPTask.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index e1d9be8ec47..6a617706a44 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -613,14 +613,6 @@ void MPPTask::runImpl() scheduleOrWait(); - auto sink = dag_context->getCTESink(); - auto source = dag_context->getCTESource(); - if (sink && source) - { - auto * log = &Poco::Logger::get("LRUCache"); - LOG_INFO(log, "xzxdebug both sink and source are set in the same mpp task"); - } - auto time_cost_in_schedule_ns = stopwatch.elapsed() - time_cost_in_preprocess_ns; dag_context->minTSO_wait_time_ns = time_cost_in_schedule_ns; auto time_cost_in_schedule_ms = time_cost_in_schedule_ns / MILLISECOND_TO_NANO; From 39209bd0d437b18c88fd2ecea64a2570e6ed95d0 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 29 Jul 2025 17:53:46 +0800 Subject: [PATCH 108/118] fix bug --- dbms/src/Operators/CTEPartition.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index e1442842021..f550f371f14 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -59,16 +59,16 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) std::unique_lock aux_lock(*(this->aux_lock)); this->total_blocks.fetch_add(1); CTEOpStatus ret_status = CTEOpStatus::OK; + if unlikely (this->status != CTEPartitionStatus::NORMAL && block.rows() != 0) + // Block memory usage will be calculated after the finish of spill + this->tmp_blocks.push_back(block); + switch (this->status) { case CTEPartitionStatus::NEED_SPILL: - ret_status = CTEOpStatus::NEED_SPILL; + return CTEOpStatus::NEED_SPILL; case CTEPartitionStatus::IN_SPILLING: - ret_status = CTEOpStatus::WAIT_SPILL; - if likely (block.rows() != 0) - // Block memory usage will be calculated after the finish of spill - this->tmp_blocks.push_back(block); - return ret_status; + return CTEOpStatus::WAIT_SPILL; case CTEPartitionStatus::NORMAL: break; } @@ -91,9 +91,10 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomic_size_t & row_num) { + // TODO remove xzxdebug LOG_INFO( this->config->log, - fmt::format("Partition {} starts cte spill for {}", this->partition_id, this->config->query_id_and_cte_id)); + fmt::format("xzxdebug Partition {} starts cte spill for {}", this->partition_id, this->config->query_id_and_cte_id)); std::unique_lock lock(*(this->mu), std::defer_lock); { std::lock_guard aux_lock(*(this->aux_lock)); From fc7611ad3f5bd8ec3765b39ce0868578450f226f Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 30 Jul 2025 12:54:46 +0800 Subject: [PATCH 109/118] tweaking --- dbms/src/Operators/CTE.cpp | 2 ++ dbms/src/Operators/CTE.h | 5 ++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 83a777aaf35..75c1871d9b5 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -64,6 +64,7 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block return CTEOpStatus::SINK_NOT_REGISTERED; auto status = this->partitions[partition_id]->tryGetBlock(cte_reader_id, block); + std::lock_guard lock(this->mu_test); switch (status) { case CTEOpStatus::OK: @@ -106,6 +107,7 @@ CTEOpStatus CTE::getBlockFromDisk(size_t cte_reader_id, size_t partition_id, Blo } auto ret = this->partitions[partition_id]->getBlockFromDisk(cte_reader_id, block); + std::lock_guard lock(this->mu_test); if (ret == CTEOpStatus::OK && block) { { diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 9b5c2ee49e7..cee5cb748a6 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -51,15 +51,14 @@ class CTE this->partitions.push_back(std::make_shared(i)); } + // TODO remove, for test + std::mutex mu_test; std::atomic_size_t total_recv_blocks = 0; std::atomic_size_t total_recv_rows = 0; - std::atomic_size_t total_spilled_blocks = 0; std::atomic_size_t total_spilled_rows = 0; - std::map total_fetch_blocks; std::map total_fetch_rows; - std::map total_fetch_blocks_in_disk; std::map total_fetch_rows_in_disk; From 5e288ad944ac8c5629177935ba2da1e12b004836 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 25 Aug 2025 17:43:52 +0800 Subject: [PATCH 110/118] fix and format --- dbms/src/Flash/Mpp/MPPTask.cpp | 6 +- .../Planner/Plans/PhysicalProjection.cpp | 56 ------------------- .../Flash/Planner/Plans/PhysicalProjection.h | 7 --- dbms/src/Operators/CTE.h | 17 ++++-- dbms/src/Operators/CTEPartition.cpp | 18 ++++-- dbms/src/Operators/CTEPartition.h | 4 +- dbms/src/Operators/CTEReader.cpp | 7 +-- dbms/src/Operators/CTESourceOp.h | 3 +- 8 files changed, 33 insertions(+), 85 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index b3736f21c56..67d1f0109f9 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -42,10 +42,6 @@ #include #include #include -<<<<<<< HEAD -======= - ->>>>>>> master namespace DB { @@ -258,7 +254,7 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) String query_id_and_cte_id = fmt::format("{}_{}", context->getDAGContext()->getMPPTaskId().getQueryID(), cte_sink.cte_id()); context->getDAGContext()->setQueryIDAndCTEIDForSink(query_id_and_cte_id); - auto cte = context->getCTEManager()->getCTE( + auto cte = context->getCTEManager()->getOrCreateCTE( query_id_and_cte_id, context->getMaxStreams(), cte_sink.cte_sink_num(), diff --git a/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp index ac47216c23e..b1d5c970ff0 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp @@ -147,62 +147,6 @@ PhysicalPlanNodePtr PhysicalProjection::buildRootFinal( return physical_projection; } -PhysicalPlanNodePtr PhysicalProjection::buildRootFinalForCTE( - const Context & context, - const LoggerPtr & log, - const PhysicalPlanNodePtr & child, - const tipb::CTESink & sink, - bool keep_session_timezone_info) -{ - RUNTIME_CHECK(child); - - const NamesAndTypes & child_schema = child->getSchema(); - DAGExpressionAnalyzer analyzer{child_schema, context}; - ExpressionActionsPtr project_actions = PhysicalPlanHelper::newActions(child->getSampleBlock()); - - auto required_schema_size = sink.field_types_size(); - std::vector required_schema; - required_schema.reserve(required_schema_size); - std::vector output_offsets; - output_offsets.reserve(required_schema_size); - for (int i = 0; i < required_schema_size; i++) - { - required_schema.push_back(sink.field_types(i)); - output_offsets.push_back(i); - } - - NamesWithAliases project_aliases - = analyzer - .buildFinalProjection(project_actions, required_schema, output_offsets, "", keep_session_timezone_info); - NamesWithAliases final_project_aliases; - auto col_num = child_schema.size(); - RUNTIME_CHECK(required_schema_size == static_cast(col_num)); - for (size_t i = 0; i < col_num; i++) - final_project_aliases.push_back(std::make_pair(project_aliases[i].first, genNameForCTESource(i))); - - project_actions->add(ExpressionAction::project(final_project_aliases)); - NamesAndTypes schema; - for (size_t i = 0; i < final_project_aliases.size(); ++i) - { - const auto & alias = final_project_aliases[i].second; - RUNTIME_CHECK(!alias.empty()); - const auto & type = analyzer.getCurrentInputColumns()[i].type; - schema.emplace_back(alias, type); - } - - auto physical_projection = std::make_shared( - child->execId(), - schema, - child->getFineGrainedShuffle(), - log->identifier(), - child, - "final projection", - project_actions); - // Final Projection is not a tidb operator, so no need to record profile streams. - physical_projection->notTiDBOperator(); - return physical_projection; -} - void PhysicalProjection::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { child->buildBlockInputStream(pipeline, context, max_streams); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalProjection.h b/dbms/src/Flash/Planner/Plans/PhysicalProjection.h index 9fcaf92328e..40bd15eb868 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalProjection.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalProjection.h @@ -51,13 +51,6 @@ class PhysicalProjection : public PhysicalUnary const PhysicalPlanNodePtr & child, Int32 cte_id = -1); - static PhysicalPlanNodePtr buildRootFinalForCTE( - const Context & context, - const LoggerPtr & log, - const PhysicalPlanNodePtr & child, - const tipb::CTESink & sink, - bool keep_session_timezone_info); - PhysicalProjection( const String & executor_id_, const NamesAndTypes & schema_, diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 0e90bd7f810..71508c1026d 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -50,7 +50,7 @@ class CTE { for (size_t i = 0; i < this->partition_num; i++) { - this->partitions.push_back(std::make_shared(i)); + this->partitions.push_back(std::make_shared(i, expected_source_num_)); this->partitions.back()->mu = std::make_unique(); this->partitions.back()->pipe_cv = std::make_unique(); } @@ -131,9 +131,15 @@ class CTE template CTEOpStatus pushBlock(size_t partition_id, const Block & block); template - void notifyEOF() { this->notifyImpl(true, ""); } + void notifyEOF() + { + this->notifyImpl(true, ""); + } template - void notifyCancel(const String & msg) { this->notifyImpl(false, msg); } + void notifyCancel(const String & msg) + { + this->notifyImpl(false, msg); + } String getError() { @@ -247,7 +253,10 @@ class CTE return this->checkBlockAvailableImpl(cte_reader_id, partition_id); } - std::shared_ptr & getPartitionForTest(size_t partition_idx) { return this->partitions[partition_idx]; } + std::shared_ptr & getPartitionForTest(size_t partition_idx) + { + return this->partitions[partition_idx]; + } private: template diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 92a5d35d47d..029cfb3d2bc 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -44,8 +44,6 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) if (!this->isBlockAvailableInMemoryNoLock(cte_reader_id)) return CTEOpStatus::BLOCK_NOT_AVAILABLE; - // TODO blocks that have been cleared should not be spilled into disk - // TODO add a variable that records the max idx block that has been cleared auto idx = this->getIdxInMemoryNoLock(cte_reader_id); block = this->blocks[idx].block; if ((--this->blocks[idx].counter) == 0) @@ -103,7 +101,10 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi // TODO remove xzxdebug LOG_INFO( this->config->log, - fmt::format("xzxdebug Partition {} starts cte spill for {}", this->partition_id, this->config->query_id_and_cte_id)); + fmt::format( + "xzxdebug Partition {} starts cte spill for {}", + this->partition_id, + this->config->query_id_and_cte_id)); std::unique_lock lock(*(this->mu), std::defer_lock); { std::lock_guard aux_lock(*(this->aux_lock)); @@ -157,11 +158,16 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi while (iter != end_iter) { - spilled_blocks.push_back(iter->block); + if (iter->counter != 0) + spilled_blocks.push_back(iter->block); ++iter; } - RUNTIME_CHECK(!spilled_blocks.empty()); + if unlikely (spilled_blocks.size() == 0) + { + ++split_iter; + continue; + } this->total_block_in_disk_num += spilled_blocks.size(); @@ -173,7 +179,7 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi spiller->spillBlocks(std::move(spilled_blocks), this->partition_id); spiller->finishSpill(); this->spillers.insert(std::make_pair(split_iter->first, std::move(spiller))); - split_iter++; + ++split_iter; } this->blocks.clear(); diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index d5af98f3d6f..173c9af788e 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -16,8 +16,8 @@ #include #include -#include #include +#include #include #include @@ -108,7 +108,7 @@ struct BlockWithCounter struct CTEPartition { - explicit CTEPartition(size_t partition_id_, size_t expected_source_num_) + CTEPartition(size_t partition_id_, size_t expected_source_num_) : partition_id(partition_id_) , aux_lock(std::make_unique()) , status(CTEPartitionStatus::NORMAL) diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index f4b3f0231e8..a5a0be6d4ab 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -36,23 +36,22 @@ CTEOpStatus CTEReader::fetchNextBlock(size_t source_id, Block & block) case CTEOpStatus::IO_IN: case CTEOpStatus::SINK_NOT_REGISTERED: case CTEOpStatus::BLOCK_NOT_AVAILABLE: + case CTEOpStatus::CANCELLED: return ret; case CTEOpStatus::OK: this->total_fetch_blocks.fetch_add(1); this->total_fetch_rows.fetch_add(block.rows()); return ret; - case CTEOpStatus::CANCELLED: - throw Exception(this->cte->getError()); // TODO cancel should not throw exception } throw Exception("Should not reach here"); } -CTEOpStatus CTEReader::fetchBlockFromDisk(size_t partition_id, Block & block) +CTEOpStatus CTEReader::fetchBlockFromDisk(size_t source_id, Block & block) { bool expected_bool = false; if (this->is_first_log.compare_exchange_weak(expected_bool, true)) LOG_INFO(this->cte->getLog(), "Begin restore data from disk for cte."); - return this->cte->getBlockFromDisk(this->cte_reader_id, partition_id, block); + return this->cte->getBlockFromDisk(this->cte_reader_id, source_id, block); } } // namespace DB diff --git a/dbms/src/Operators/CTESourceOp.h b/dbms/src/Operators/CTESourceOp.h index a5ae9e50d05..16c8c13ea1d 100644 --- a/dbms/src/Operators/CTESourceOp.h +++ b/dbms/src/Operators/CTESourceOp.h @@ -58,11 +58,12 @@ class CTESourceOp : public SourceOp const NamesAndTypes & schema, const String & query_id_and_cte_id_) : SourceOp(exec_context_, req_id) + , query_id_and_cte_id(query_id_and_cte_id_) , cte_reader(cte_reader_) , io_profile_info(IOProfileInfo::createForRemote(profile_info_ptr, 1)) , id(id_) , notifier(this->cte_reader->getCTE(), this->cte_reader->getID(), this->id) - , query_id_and_cte_id(query_id_and_cte_id_) + , io_notifier(this->cte_reader->getCTE(), this->id) { setHeader(Block(getColumnWithTypeAndName(schema))); } From dc65e964dd5e16a4416ffeac8fdcf3d07e9b456f Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 25 Aug 2025 18:42:14 +0800 Subject: [PATCH 111/118] refine --- dbms/src/Common/TiFlashMetrics.h | 3 ++- .../Schedule/Tasks/PipeConditionVariable.h | 3 +++ dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 1 + dbms/src/Operators/CTE.cpp | 8 ++++---- dbms/src/Operators/CTE.h | 9 +++++---- dbms/src/Operators/CTEPartition.cpp | 19 ++++++++++--------- dbms/src/Operators/CTEPartition.h | 4 ++++ dbms/src/Operators/CTESourceOp.cpp | 2 +- 8 files changed, 30 insertions(+), 19 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 3f3c47246e4..c460b1085a7 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -778,7 +778,8 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_wait_on_join_build, {"type", "wait_on_join_build"}), \ F(type_wait_on_join_probe, {"type", "wait_on_join_probe"}), \ F(type_wait_on_result_queue_write, {"type", "wait_on_result_queue_write"}), \ - F(type_type_wait_on_cte_read, {"type", "type_wait_on_cte_read"})) \ + F(type_type_wait_on_cte_read, {"type", "type_wait_on_cte_read"}), \ + F(type_type_wait_on_cte_io, {"type", "type_wait_on_cte_io"})) \ M(tiflash_pipeline_task_duration_seconds, \ "Bucketed histogram of pipeline task duration in seconds", \ Histogram, /* these command usually cost several hundred milliseconds to several seconds, increase the start bucket to 5ms */ \ diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index a17fe2d4ab5..2871699958b 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -60,6 +60,9 @@ class PipeConditionVariable case NotifyType::WAIT_ON_CTE_READ: GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_type_wait_on_cte_read).Increment(change); break; + case NotifyType::WAIT_ON_CTE_IO: + GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_type_wait_on_cte_io).Increment(change); + break; case NotifyType::WAIT_ON_NOTHING: throw Exception("task notify type should be set before register or notify"); break; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h index 35103787b90..8b525b662c1 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -59,6 +59,7 @@ enum class NotifyType WAIT_ON_JOIN_PROBE_FINISH, WAIT_ON_RESULT_QUEUE_WRITE, WAIT_ON_CTE_READ, + WAIT_ON_CTE_IO, WAIT_ON_NOTHING, }; diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 67d803d38e9..4656c59142a 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -63,9 +63,6 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block if unlikely (!this->areAllSinksRegistered()) return CTEOpStatus::SINK_NOT_REGISTERED; - if unlikely (block.rows() == 0) - return CTEOpStatus::OK; - auto status = this->partitions[partition_id]->tryGetBlock(cte_reader_id, block); std::lock_guard lock(this->mu_test); switch (status) @@ -110,6 +107,8 @@ CTEOpStatus CTE::getBlockFromDisk(size_t cte_reader_id, size_t partition_id, Blo } auto ret = this->partitions[partition_id]->getBlockFromDisk(cte_reader_id, block); + // --------------------- + // TODO delete it std::lock_guard lock(this->mu_test); if (ret == CTEOpStatus::OK && block) { @@ -122,6 +121,7 @@ CTEOpStatus CTE::getBlockFromDisk(size_t cte_reader_id, size_t partition_id, Blo iter->second.fetch_add(block.rows()); } } + // --------------------- return ret; } @@ -174,7 +174,7 @@ void CTE::checkInSpillingAndRegisterTask(TaskPtr && task, size_t partition_id) std::lock_guard aux_lock(*(this->partitions[partition_id]->aux_lock)); if (this->partitions[partition_id]->status == CTEPartitionStatus::IN_SPILLING) - this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE_READ); + this->registerTask(partition_id, std::move(task), NotifyType::WAIT_ON_CTE_IO); else this->notifyTaskDirectly(partition_id, std::move(task)); } diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 71508c1026d..f2b4d5912d2 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -65,6 +65,7 @@ class CTE } } + // ------------------------ // TODO remove, for test std::mutex mu_test; std::atomic_size_t total_recv_blocks = 0; @@ -75,6 +76,7 @@ class CTE std::map total_fetch_rows; std::map total_fetch_blocks_in_disk; std::map total_fetch_rows_in_disk; + // ------------------------ ~CTE() { @@ -274,11 +276,10 @@ class CTE if unlikely (this->is_cancelled) return CTEOpStatus::CANCELLED; - if (this->partitions[partition_id]->blocks.size() - <= this->partitions[partition_id]->fetch_block_idxs[cte_reader_id]) - return this->is_eof ? CTEOpStatus::END_OF_FILE : CTEOpStatus::BLOCK_NOT_AVAILABLE; + if (this->partitions[partition_id]->isBlockAvailableInMemoryNoLock(cte_reader_id)) + return CTEOpStatus::OK; - return CTEOpStatus::OK; + return this->is_eof ? CTEOpStatus::END_OF_FILE : CTEOpStatus::BLOCK_NOT_AVAILABLE; } Int32 getTotalExitNumNoLock() const noexcept { return this->sink_exit_num + this->source_exit_num; } diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 029cfb3d2bc..2ebaffa8984 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -46,9 +46,12 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) auto idx = this->getIdxInMemoryNoLock(cte_reader_id); block = this->blocks[idx].block; + assert(this->blocks[idx].counter > 0); + if ((--this->blocks[idx].counter) == 0) this->blocks[idx].block.clear(); { + // TODO delete it auto [iter, _] = this->fetch_in_mem_idxs.insert(std::make_pair(cte_reader_id, 0)); iter->second.push_back(this->fetch_block_idxs[cte_reader_id]); } @@ -78,7 +81,7 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) // mu must be held after aux_lock so that we will not be blocked by spill. // Blocked in cpu pool is very bad. - std::lock_guard lock(*this->mu); + std::lock_guard lock(*(this->mu)); this->memory_usage += block.bytes(); this->blocks.push_back(BlockWithCounter(block, static_cast(this->expected_source_num))); @@ -124,11 +127,10 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi } // Key represents logical index - // Value represents physical index at `this->blocks` + // Value represents physical index in `this->blocks` std::map split_idxs; - split_idxs.insert(std::make_pair(this->total_block_in_disk_num, 0)); for (const auto & [_, logical_idx] : this->fetch_block_idxs) - if (logical_idx > this->total_block_in_disk_num) + if (logical_idx >= this->total_block_in_disk_num) split_idxs.insert(std::make_pair(logical_idx, logical_idx - this->total_block_in_disk_num)); auto blocks_begin_iter = this->blocks.begin(); @@ -163,19 +165,18 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi ++iter; } - if unlikely (spilled_blocks.size() == 0) - { - ++split_iter; - continue; - } + RUNTIME_CHECK(spilled_blocks.size() == 0); this->total_block_in_disk_num += spilled_blocks.size(); auto spiller = this->config->getSpiller(this->partition_id, this->spillers.size()); + // ----------------- + // TODO delete it this->total_spill_blocks.fetch_add(spilled_blocks.size()); block_num.fetch_add(spilled_blocks.size()); for (auto & block : spilled_blocks) row_num.fetch_add(block.rows()); + // ----------------- spiller->spillBlocks(std::move(spilled_blocks), this->partition_id); spiller->finishSpill(); this->spillers.insert(std::make_pair(split_iter->first, std::move(spiller))); diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 173c9af788e..cd2f358da67 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -191,17 +191,21 @@ struct CTEPartition tmp_blocks.clear(); } + // ----------- + // TODO delete them std::atomic_size_t total_blocks = 0; std::atomic_size_t total_spill_blocks = 0; std::vector> spill_ranges; std::map> fetch_in_mem_idxs; std::map> fetch_in_disk_idxs; + // ----------- size_t total_byte_usage = 0; size_t partition_id; + // Protect `status` and `tmp_blocks` variables std::unique_ptr aux_lock; CTEPartitionStatus status; Blocks tmp_blocks; diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index a448178c87e..c2dd0bc508b 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -28,7 +28,7 @@ void CTESourceOp::operateSuffixImpl() OperatorStatus CTESourceOp::readImpl(Block & block) { - if (this->block_from_disk) + if unlikely (this->block_from_disk) { block = this->block_from_disk; this->block_from_disk.clear(); From c01971210211f8f211fd8223582e5c3652f1e723 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 2 Sep 2025 15:50:46 +0800 Subject: [PATCH 112/118] fix bugs --- dbms/src/Interpreters/CTESpillContext.cpp | 11 +-- dbms/src/Operators/CTE.h | 8 +- dbms/src/Operators/CTEPartition.cpp | 90 ++++++++++++++++++----- dbms/src/Operators/CTEPartition.h | 23 +++++- dbms/src/Operators/CTEReader.cpp | 13 +++- dbms/src/Operators/CTEReader.h | 10 ++- 6 files changed, 118 insertions(+), 37 deletions(-) diff --git a/dbms/src/Interpreters/CTESpillContext.cpp b/dbms/src/Interpreters/CTESpillContext.cpp index bd24af88f9a..4601a1654e5 100644 --- a/dbms/src/Interpreters/CTESpillContext.cpp +++ b/dbms/src/Interpreters/CTESpillContext.cpp @@ -21,9 +21,6 @@ namespace DB { Int64 CTESpillContext::triggerSpillImpl(Int64 expected_released_memories) { - String info = "xzxdebug CTE auto spill is triggered, spilled partitions: "; - auto * log = &Poco::Logger::get("LRUCache"); - for (auto & partition : this->partitions) { std::lock_guard aux_lock(*(partition->aux_lock)); @@ -31,17 +28,15 @@ Int64 CTESpillContext::triggerSpillImpl(Int64 expected_released_memories) continue; partition->status = CTEPartitionStatus::NEED_SPILL; - info = fmt::format("{}, {}", info, partition->partition_id); std::lock_guard lock(*(partition->mu)); + if (partition->memory_usage == 0) + continue; + expected_released_memories = std::max(expected_released_memories - partition->memory_usage, 0); if (expected_released_memories <= 0) - { - LOG_INFO(log, info); return expected_released_memories; - } } - LOG_INFO(log, info); return expected_released_memories; } } // namespace DB diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index f2b4d5912d2..1c3c8854bab 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -51,6 +51,8 @@ class CTE for (size_t i = 0; i < this->partition_num; i++) { this->partitions.push_back(std::make_shared(i, expected_source_num_)); + for (size_t cte_reader_id = 0; cte_reader_id < expected_source_num_; cte_reader_id++) + this->partitions.back()->fetch_block_idxs.insert(std::make_pair(cte_reader_id, 0)); this->partitions.back()->mu = std::make_unique(); this->partitions.back()->pipe_cv = std::make_unique(); } @@ -122,11 +124,7 @@ class CTE "next_cte_reader_id: {}, expected_source_num: {}", this->next_cte_reader_id, this->expected_source_num); - auto cte_reader_id = this->next_cte_reader_id; - this->next_cte_reader_id++; - for (auto & item : this->partitions) - item->fetch_block_idxs.insert(std::make_pair(cte_reader_id, 0)); - return cte_reader_id; + return this->next_cte_reader_id++; } CTEOpStatus tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block & block); diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 2ebaffa8984..3fd456c9af3 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -24,8 +24,15 @@ namespace DB { size_t CTEPartition::getIdxInMemoryNoLock(size_t cte_reader_id) { - RUNTIME_CHECK(this->fetch_block_idxs[cte_reader_id] >= this->total_block_in_disk_num); - return this->fetch_block_idxs[cte_reader_id] - this->total_block_in_disk_num; + auto idx = this->fetch_block_idxs[cte_reader_id]; + auto total_evicted = this->getTotalEvictedBlockNumnoLock(); + RUNTIME_CHECK_MSG( + idx >= total_evicted, + "partition id: {}, idx: {}, total_evicted: {}", + this->partition_id, + idx, + total_evicted); + return idx - total_evicted; } CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) @@ -34,7 +41,7 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) if (this->status == CTEPartitionStatus::IN_SPILLING) return CTEOpStatus::WAIT_SPILL; - std::lock_guard lock(*this->mu); + std::lock_guard lock(*(this->mu)); this->putTmpBlocksIntoBlocksNoLock(); @@ -101,13 +108,6 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomic_size_t & row_num) { - // TODO remove xzxdebug - LOG_INFO( - this->config->log, - fmt::format( - "xzxdebug Partition {} starts cte spill for {}", - this->partition_id, - this->config->query_id_and_cte_id)); std::unique_lock lock(*(this->mu), std::defer_lock); { std::lock_guard aux_lock(*(this->aux_lock)); @@ -126,15 +126,39 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi this->putTmpBlocksIntoBlocksNoLock(); } + if (this->first_log) + { + // TODO remove xzxdebug + LOG_INFO( + this->config->log, + fmt::format( + "xzxdebug Partition {} starts cte spill for {}", + this->partition_id, + this->config->query_id_and_cte_id)); + this->first_log = false; + } + + // auto * log = &Poco::Logger::get("LRUCache"); + String info = fmt::format("xzxdebug spill detail {} ", this->partition_id); + // Key represents logical index // Value represents physical index in `this->blocks` std::map split_idxs; - for (const auto & [_, logical_idx] : this->fetch_block_idxs) - if (logical_idx >= this->total_block_in_disk_num) - split_idxs.insert(std::make_pair(logical_idx, logical_idx - this->total_block_in_disk_num)); + auto evicted_block_num = this->getTotalEvictedBlockNumnoLock(); + split_idxs.insert(std::make_pair(evicted_block_num, 0)); + info = fmt::format("{} ", info, evicted_block_num, 0); + for (const auto & [cte_reader_id, logical_idx] : this->fetch_block_idxs) + { + info = fmt::format("{} <{}:{}>", info, cte_reader_id, logical_idx); + if (logical_idx > evicted_block_num) + { + split_idxs.insert(std::make_pair(logical_idx, logical_idx - evicted_block_num)); + info = fmt::format("{} ", info, logical_idx, logical_idx - evicted_block_num); + } + } - auto blocks_begin_iter = this->blocks.begin(); auto split_iter = split_idxs.begin(); + auto blocks_begin_iter = this->blocks.begin(); auto total_block_in_memory_num = this->blocks.size(); while (split_iter != split_idxs.end()) { @@ -158,15 +182,44 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi end_iter = blocks_begin_iter + next_iter->second; } + info = fmt::format("{} ", info, spill_ranges.back().first, spill_ranges.back().second); + + bool counter_is_zero = false; + if (iter->counter == 0) + // In one slice, all blocks' counter should be 0 or not be 0. Check it. + counter_is_zero = true; + + // TODO remove it + auto before_release_num = this->total_block_released_num; + while (iter != end_iter) { - if (iter->counter != 0) + if (counter_is_zero) + { + RUNTIME_CHECK(iter->counter == 0); + this->total_block_released_num++; + } + else + { + RUNTIME_CHECK(iter->counter != 0); spilled_blocks.push_back(iter->block); + } ++iter; } - RUNTIME_CHECK(spilled_blocks.size() == 0); + // TODO remove + if (before_release_num != this->total_block_released_num) + info = fmt::format("{} released_num: {}->{}", info, before_release_num, this->total_block_released_num); + + if (counter_is_zero) + { + split_iter = next_iter; + continue; + } + + RUNTIME_CHECK(spilled_blocks.size() != 0); + // LOG_INFO(log, "xzxdebug spill info {} total_block_in_disk_num {}, spilled_blocks.size(): {}, total_block_in_disk_num change: {}->{}", this->partition_id, this->total_block_in_disk_num, spilled_blocks.size(), this->total_block_in_disk_num, this->total_block_in_disk_num+spilled_blocks.size()); this->total_block_in_disk_num += spilled_blocks.size(); auto spiller = this->config->getSpiller(this->partition_id, this->spillers.size()); @@ -180,9 +233,12 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi spiller->spillBlocks(std::move(spilled_blocks), this->partition_id); spiller->finishSpill(); this->spillers.insert(std::make_pair(split_iter->first, std::move(spiller))); - ++split_iter; + split_iter = next_iter; } + // if (this->blocks.size() > 0) + // LOG_INFO(log, "xzxdebug block change {} {} -> 0", this->partition_id, this->blocks.size()); + // LOG_INFO(log, info); this->blocks.clear(); this->memory_usage = 0; diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index cd2f358da67..b0e8daf62d3 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -148,17 +149,31 @@ struct CTEPartition void setSharedConfig(std::shared_ptr config) { this->config = config; } size_t getIdxInMemoryNoLock(size_t cte_reader_id); + + UInt64 getTotalEvictedBlockNumnoLock() const + { + return this->total_block_released_num + this->total_block_in_disk_num; + } + bool isBlockAvailableInDiskNoLock(size_t cte_reader_id) { - return this->fetch_block_idxs[cte_reader_id] < this->total_block_in_disk_num; + auto idx = this->fetch_block_idxs[cte_reader_id]; + RUNTIME_CHECK_MSG( + idx >= this->total_block_released_num, + "partition: {}, idx: {}, total_block_released_num: {}", + this->partition_id, + idx, + this->total_block_released_num); + return idx < this->getTotalEvictedBlockNumnoLock(); } + bool isBlockAvailableInMemoryNoLock(size_t cte_reader_id) { return this->getIdxInMemoryNoLock(cte_reader_id) < this->blocks.size(); } bool isSpillTriggeredNoLock() const { return this->total_block_in_disk_num > 0; } - void addIdxNoLock(size_t cte_reader_id) { this->fetch_block_idxs[cte_reader_id]++; } + void addIdxNoLock(size_t cte_reader_id) { ++this->fetch_block_idxs[cte_reader_id]; } bool exceedMemoryThresholdNoLock() const { if (this->config->memory_threshold == 0) @@ -199,6 +214,8 @@ struct CTEPartition std::vector> spill_ranges; std::map> fetch_in_mem_idxs; std::map> fetch_in_disk_idxs; + + bool first_log = true; // ----------- size_t total_byte_usage = 0; @@ -218,10 +235,10 @@ struct CTEPartition size_t memory_usage = 0; const size_t expected_source_num; - std::vector block_in_disk_nums; std::unordered_map spillers; std::unordered_map cte_reader_restore_streams; UInt64 total_block_in_disk_num = 0; + UInt64 total_block_released_num = 0; std::shared_ptr config; diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index a5a0be6d4ab..6997b68802f 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -37,10 +37,13 @@ CTEOpStatus CTEReader::fetchNextBlock(size_t source_id, Block & block) case CTEOpStatus::SINK_NOT_REGISTERED: case CTEOpStatus::BLOCK_NOT_AVAILABLE: case CTEOpStatus::CANCELLED: + // TODO merge return ret; case CTEOpStatus::OK: + // TODO remove this->total_fetch_blocks.fetch_add(1); this->total_fetch_rows.fetch_add(block.rows()); + this->total_fetch_from_mem.fetch_add(1); return ret; } throw Exception("Should not reach here"); @@ -51,7 +54,13 @@ CTEOpStatus CTEReader::fetchBlockFromDisk(size_t source_id, Block & block) bool expected_bool = false; if (this->is_first_log.compare_exchange_weak(expected_bool, true)) LOG_INFO(this->cte->getLog(), "Begin restore data from disk for cte."); - - return this->cte->getBlockFromDisk(this->cte_reader_id, source_id, block); + auto ret = this->cte->getBlockFromDisk(this->cte_reader_id, source_id, block); + if (block) + { + this->total_fetch_blocks.fetch_add(1); + this->total_fetch_rows.fetch_add(block.rows()); + this->total_fetch_from_disk.fetch_add(1); + } + return ret; } } // namespace DB diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index dd60dc35300..9bf01a52805 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -44,6 +44,9 @@ class CTEReader std::atomic_size_t total_fetch_blocks = 0; std::atomic_size_t total_fetch_rows = 0; + std::atomic_size_t total_fetch_from_mem = 0; + std::atomic_size_t total_fetch_from_disk = 0; + // For Test CTEReader(const String & query_id_and_cte_id_, CTEManager * cte_manager_, std::shared_ptr cte_) : query_id_and_cte_id(query_id_and_cte_id_) @@ -62,9 +65,12 @@ class CTEReader LOG_INFO( log, fmt::format( - "xzxdebug CTEReader fb: {} fr: {}", + "xzxdebug CTEReader {} fb: {} fr: {}, from mem: {}, from disk: {}", + this->cte_reader_id, this->total_fetch_blocks.load(), - this->total_fetch_rows.load())); + this->total_fetch_rows.load(), + this->total_fetch_from_mem.load(), + this->total_fetch_from_disk.load())); } CTEOpStatus fetchNextBlock(size_t source_id, Block & block); From b97b49ac8f3780d444a11e90ecba3f4b6ed7d7f1 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 4 Sep 2025 16:45:07 +0800 Subject: [PATCH 113/118] fix ut --- dbms/src/Operators/CTEPartition.h | 4 ++++ dbms/src/Operators/CTEReader.cpp | 22 ++++++++++++++++++++++ dbms/src/Operators/tests/gtest_cte.cpp | 5 ++++- 3 files changed, 30 insertions(+), 1 deletion(-) diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index b0e8daf62d3..7810af2674a 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -176,6 +176,10 @@ struct CTEPartition void addIdxNoLock(size_t cte_reader_id) { ++this->fetch_block_idxs[cte_reader_id]; } bool exceedMemoryThresholdNoLock() const { + // config will be nullptr in test + if unlikely (this->config == nullptr) + return false; + if (this->config->memory_threshold == 0) return false; return this->memory_usage >= this->config->memory_threshold; diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 6997b68802f..347dab4f38b 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -63,4 +63,26 @@ CTEOpStatus CTEReader::fetchBlockFromDisk(size_t source_id, Block & block) } return ret; } + +CTEOpStatus CTEReader::waitForBlockAvailableForTest(size_t partition_idx) +{ + auto & partition = this->cte->getPartitionForTest(partition_idx); + std::unique_lock lock(*(partition->mu_for_test)); + while (true) + { + partition->cv_for_test->wait(lock); + auto status = this->cte->checkBlockAvailable(this->cte_reader_id, partition_idx); + switch (status) + { + case CTEOpStatus::BLOCK_NOT_AVAILABLE: + break; + case CTEOpStatus::OK: + case CTEOpStatus::CANCELLED: + case CTEOpStatus::END_OF_FILE: + return status; + default: + throw Exception("Should not reach here"); + } + } +} } // namespace DB diff --git a/dbms/src/Operators/tests/gtest_cte.cpp b/dbms/src/Operators/tests/gtest_cte.cpp index 7de68025632..3ba6cf47e02 100644 --- a/dbms/src/Operators/tests/gtest_cte.cpp +++ b/dbms/src/Operators/tests/gtest_cte.cpp @@ -92,7 +92,7 @@ try size_t row_num = di(dre); Blocks sink_blocks = generateBlocks(0, row_num); for (size_t i = 0; i < sink_blocks.size(); i++) - ASSERT_TRUE(cte->pushBlock(i % PARTITION_NUM, sink_blocks[i])); + ASSERT_EQ(CTEOpStatus::OK, cte->pushBlock(i % PARTITION_NUM, sink_blocks[i])); Blocks received_blocks; std::vector received_results; @@ -219,7 +219,10 @@ void concurrentTest() { std::lock_guard lock(*(source_mus[source_idx])); received_blocks[source_idx].push_back(block); + break; } + default: + throw Exception("Should not reach here"); } } if (partition_idx == 0) From 9e514e104a4423f42aef5e13b8ef30db24248762 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 5 Sep 2025 09:50:26 +0800 Subject: [PATCH 114/118] clean codes --- .../DataStreams/NativeBlockOutputStream.cpp | 8 -------- .../src/DataStreams/NativeBlockOutputStream.h | 1 - dbms/src/Flash/Mpp/MPPTask.cpp | 20 ------------------- dbms/src/Flash/Mpp/MPPTaskId.h | 3 +-- dbms/src/Interpreters/CTESpillContext.h | 3 +-- dbms/src/Interpreters/Context.cpp | 4 ---- dbms/src/Operators/CTE.cpp | 10 ++++++---- dbms/src/Operators/CTE.h | 3 +++ dbms/src/Operators/CTEPartition.cpp | 14 ++++++++----- dbms/src/Operators/CTEReader.h | 3 ++- 10 files changed, 22 insertions(+), 47 deletions(-) diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index 98a9d08cc7e..960d7a7cb8b 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -140,12 +140,4 @@ void NativeBlockOutputStream::write(const Block & block) } } } - -Int64 NativeBlockOutputStream::writeAndReturnBlockSize(const Block & block) -{ - Int64 prev_byte_count = this->ostr.count(); - this->write(block); - return this->ostr.count() - prev_byte_count; -} - } // namespace DB diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.h b/dbms/src/DataStreams/NativeBlockOutputStream.h index 0fdc9b3bff2..9aa04e3c3b4 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.h +++ b/dbms/src/DataStreams/NativeBlockOutputStream.h @@ -44,7 +44,6 @@ class NativeBlockOutputStream : public IBlockOutputStream Block getHeader() const override { return header; } void write(const Block & block) override; - Int64 writeAndReturnBlockSize(const Block & block); void flush() override; static void writeData( diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 67d1f0109f9..f7194efbde2 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -243,26 +243,6 @@ void MPPTask::run() void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) { - if unlikely (!dag_context->dag_request.rootExecutor().has_exchange_sender()) - { - RUNTIME_CHECK_MSG( - dag_context->dag_request.rootExecutor().has_cte_sink(), - "Task should has either exchange sender or cte sink"); - - this->has_cte_sink.store(true); - const auto & cte_sink = dag_context->dag_request.rootExecutor().cte_sink(); - String query_id_and_cte_id - = fmt::format("{}_{}", context->getDAGContext()->getMPPTaskId().getQueryID(), cte_sink.cte_id()); - context->getDAGContext()->setQueryIDAndCTEIDForSink(query_id_and_cte_id); - auto cte = context->getCTEManager()->getOrCreateCTE( - query_id_and_cte_id, - context->getMaxStreams(), - cte_sink.cte_sink_num(), - cte_sink.cte_source_num()); - cte->registerSink(); - context->getDAGContext()->setCTESink(cte); - return; - } auto tunnel_set_local = std::make_shared(log->identifier()); std::chrono::seconds timeout(task_request.timeout()); const auto & exchange_sender = dag_context->dag_request.rootExecutor().exchange_sender(); diff --git a/dbms/src/Flash/Mpp/MPPTaskId.h b/dbms/src/Flash/Mpp/MPPTaskId.h index 564cdf1ac10..5ab89a49661 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.h +++ b/dbms/src/Flash/Mpp/MPPTaskId.h @@ -116,8 +116,7 @@ struct MPPGatherId query_id.query_ts, query_id.local_query_id, query_id.server_id, - query_id.start_ts, - this->gather_id); + query_id.start_ts); } bool hasMeaningfulGatherId() const { return gather_id > 0; } diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h index d280d74d49b..54d13a4d347 100644 --- a/dbms/src/Interpreters/CTESpillContext.h +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include @@ -22,8 +23,6 @@ #include #include -#include "Common/Logger.h" - namespace DB { class CTESpillContext final : public OperatorSpillContext diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 963d50215c6..513dc8b758c 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -29,7 +29,6 @@ #include #include #include -#include #include #include #include @@ -230,15 +229,12 @@ struct ContextShared std::shared_ptr shared_block_schemas; - std::unique_ptr cte_manager; - ContextShared( std::shared_ptr runtime_components_factory_, Context::ApplicationType app_type) : runtime_components_factory(std::move(runtime_components_factory_)) , storage_run_mode(PageStorageRunMode::ONLY_V3) , application_type(app_type) - , cte_manager(std::make_unique()) { /// TODO: make it singleton (?) #ifndef MULTIPLE_CONTEXT_GTEST diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index 4656c59142a..ace949a2f60 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -68,14 +68,16 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block switch (status) { case CTEOpStatus::OK: - { - auto [iter, _] = this->total_fetch_blocks.insert(std::make_pair(cte_reader_id, 0)); - iter->second.fetch_add(1); - } + // TODO delete --------------------- + { + auto [iter, _] = this->total_fetch_blocks.insert(std::make_pair(cte_reader_id, 0)); + iter->second.fetch_add(1); + } { auto [iter, _] = this->total_fetch_rows.insert(std::make_pair(cte_reader_id, 0)); iter->second.fetch_add(block.rows()); } + // --------------------- return status; case CTEOpStatus::BLOCK_NOT_AVAILABLE: return this->is_eof ? CTEOpStatus::END_OF_FILE : CTEOpStatus::BLOCK_NOT_AVAILABLE; diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 1c3c8854bab..4ef4c61a86c 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -32,6 +32,7 @@ namespace DB { +// TODO delete inline String genInfo(const String & name, const std::map & data) { String info = fmt::format("{}: ", name); @@ -82,6 +83,7 @@ class CTE ~CTE() { + // TODO delete --------------- String info; info = fmt::format( "total_recv_blocks: {}, total_recv_rows: {}, total_spilled_blocks: {}, total_spilled_rows: {}, ", @@ -99,6 +101,7 @@ class CTE for (auto & p : this->partitions) p->debugOutput(); + // TODO --------------- } void initCTESpillContextAndPartitionConfig( diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 3fd456c9af3..e4e932bed7e 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -57,11 +57,12 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) if ((--this->blocks[idx].counter) == 0) this->blocks[idx].block.clear(); + // TODO delete ------------- { - // TODO delete it auto [iter, _] = this->fetch_in_mem_idxs.insert(std::make_pair(cte_reader_id, 0)); iter->second.push_back(this->fetch_block_idxs[cte_reader_id]); } + // ------------- this->addIdxNoLock(cte_reader_id); return CTEOpStatus::OK; } @@ -128,7 +129,7 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi if (this->first_log) { - // TODO remove xzxdebug + // TODO remove LOG_INFO( this->config->log, fmt::format( @@ -138,6 +139,7 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi this->first_log = false; } + // TODO remove // auto * log = &Poco::Logger::get("LRUCache"); String info = fmt::format("xzxdebug spill detail {} ", this->partition_id); @@ -217,14 +219,14 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi continue; } - RUNTIME_CHECK(spilled_blocks.size() != 0); + RUNTIME_CHECK(!spilled_blocks.empty()); + // TODO remove // LOG_INFO(log, "xzxdebug spill info {} total_block_in_disk_num {}, spilled_blocks.size(): {}, total_block_in_disk_num change: {}->{}", this->partition_id, this->total_block_in_disk_num, spilled_blocks.size(), this->total_block_in_disk_num, this->total_block_in_disk_num+spilled_blocks.size()); this->total_block_in_disk_num += spilled_blocks.size(); auto spiller = this->config->getSpiller(this->partition_id, this->spillers.size()); - // ----------------- - // TODO delete it + // TODO delete ----------------- this->total_spill_blocks.fetch_add(spilled_blocks.size()); block_num.fetch_add(spilled_blocks.size()); for (auto & block : spilled_blocks) @@ -292,10 +294,12 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) continue; } + // TODO delete ------------- { auto [iter, _] = this->fetch_in_disk_idxs.insert(std::make_pair(cte_reader_id, 0)); iter->second.push_back(this->fetch_block_idxs[cte_reader_id]); } + // ------------- this->addIdxNoLock(cte_reader_id); break; }; diff --git a/dbms/src/Operators/CTEReader.h b/dbms/src/Operators/CTEReader.h index 9bf01a52805..00ec3a4c454 100644 --- a/dbms/src/Operators/CTEReader.h +++ b/dbms/src/Operators/CTEReader.h @@ -40,12 +40,13 @@ class CTEReader RUNTIME_CHECK(cte); } - // TODO maybe remove it + // TODO remove -------------- std::atomic_size_t total_fetch_blocks = 0; std::atomic_size_t total_fetch_rows = 0; std::atomic_size_t total_fetch_from_mem = 0; std::atomic_size_t total_fetch_from_disk = 0; + // -------------- // For Test CTEReader(const String & query_id_and_cte_id_, CTEManager * cte_manager_, std::shared_ptr cte_) From 87743d933425a4aad5e93a0d236331c6487ab418 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Fri, 5 Sep 2025 11:50:25 +0800 Subject: [PATCH 115/118] refine --- dbms/src/Common/TiFlashMetrics.h | 4 +- .../Pipeline/Schedule/Reactor/WaitReactor.cpp | 3 +- .../Schedule/Tasks/PipeConditionVariable.h | 4 +- dbms/src/Interpreters/CTESpillContext.cpp | 26 ++++++++--- dbms/src/Interpreters/CTESpillContext.h | 5 +-- dbms/src/Operators/CTE.cpp | 11 +++-- dbms/src/Operators/CTEPartition.cpp | 43 +++++-------------- dbms/src/Operators/CTEPartition.h | 8 ++-- dbms/src/Operators/CTEReader.cpp | 1 + 9 files changed, 50 insertions(+), 55 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index c460b1085a7..0724d32910b 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -778,8 +778,8 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_wait_on_join_build, {"type", "wait_on_join_build"}), \ F(type_wait_on_join_probe, {"type", "wait_on_join_probe"}), \ F(type_wait_on_result_queue_write, {"type", "wait_on_result_queue_write"}), \ - F(type_type_wait_on_cte_read, {"type", "type_wait_on_cte_read"}), \ - F(type_type_wait_on_cte_io, {"type", "type_wait_on_cte_io"})) \ + F(type_wait_on_cte_read, {"type", "type_wait_on_cte_read"}), \ + F(type_wait_on_cte_io, {"type", "type_wait_on_cte_io"})) \ M(tiflash_pipeline_task_duration_seconds, \ "Bucketed histogram of pipeline task duration in seconds", \ Histogram, /* these command usually cost several hundred milliseconds to several seconds, increase the start bucket to 5ms */ \ diff --git a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp index 31724125e36..6dfe93cfaef 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp @@ -40,7 +40,8 @@ WaitReactor::WaitReactor(TaskScheduler & scheduler_) GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_join_probe).Set(0); GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_result_queue_write).Set(0); GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_shared_queue_write).Set(0); - GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_type_wait_on_cte_read).Set(0); + GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte_read).Set(0); + GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte_io).Set(0); thread = std::thread(&WaitReactor::loop, this); } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index 2871699958b..9904aa1ae98 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -58,10 +58,10 @@ class PipeConditionVariable GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_shared_queue_write).Increment(change); break; case NotifyType::WAIT_ON_CTE_READ: - GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_type_wait_on_cte_read).Increment(change); + GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte_read).Increment(change); break; case NotifyType::WAIT_ON_CTE_IO: - GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_type_wait_on_cte_io).Increment(change); + GET_METRIC(tiflash_pipeline_wait_on_notify_tasks, type_wait_on_cte_io).Increment(change); break; case NotifyType::WAIT_ON_NOTHING: throw Exception("task notify type should be set before register or notify"); diff --git a/dbms/src/Interpreters/CTESpillContext.cpp b/dbms/src/Interpreters/CTESpillContext.cpp index 4601a1654e5..c4cce759319 100644 --- a/dbms/src/Interpreters/CTESpillContext.cpp +++ b/dbms/src/Interpreters/CTESpillContext.cpp @@ -15,25 +15,39 @@ #include #include +#include #include +#include namespace DB { Int64 CTESpillContext::triggerSpillImpl(Int64 expected_released_memories) { - for (auto & partition : this->partitions) + size_t partition_num = this->partitions.size(); + std::vector> part_idx_with_mem_usages; + part_idx_with_mem_usages.reserve(partition_num); + for (size_t i = 0; i < partition_num; i++) + part_idx_with_mem_usages.push_back(std::make_pair(this->partitions[i]->memory_usage.load(), i)); + + std::sort( + part_idx_with_mem_usages.begin(), + part_idx_with_mem_usages.end(), + [](const std::pair & l, const std::pair & r) { return l.first > r.first; }); + + for (const auto & item : part_idx_with_mem_usages) { + auto memory_usage = item.first; + const std::shared_ptr & partition = partitions[item.second]; std::lock_guard aux_lock(*(partition->aux_lock)); if (partition->status != CTEPartitionStatus::NORMAL) continue; - partition->status = CTEPartitionStatus::NEED_SPILL; - - std::lock_guard lock(*(partition->mu)); - if (partition->memory_usage == 0) + if (memory_usage == 0) continue; - expected_released_memories = std::max(expected_released_memories - partition->memory_usage, 0); + partition->status = CTEPartitionStatus::NEED_SPILL; + + expected_released_memories = std::max(expected_released_memories - memory_usage, 0); if (expected_released_memories <= 0) return expected_released_memories; } diff --git a/dbms/src/Interpreters/CTESpillContext.h b/dbms/src/Interpreters/CTESpillContext.h index 54d13a4d347..12bb3f81dca 100644 --- a/dbms/src/Interpreters/CTESpillContext.h +++ b/dbms/src/Interpreters/CTESpillContext.h @@ -45,10 +45,7 @@ class CTESpillContext final : public OperatorSpillContext { Int64 total_memory = 0; for (auto & partition : this->partitions) - { - std::lock_guard lock(*(partition->mu)); - total_memory += partition->memory_usage; - } + total_memory += partition->memory_usage.load(); return total_memory; } diff --git a/dbms/src/Operators/CTE.cpp b/dbms/src/Operators/CTE.cpp index ace949a2f60..48b42d561ac 100644 --- a/dbms/src/Operators/CTE.cpp +++ b/dbms/src/Operators/CTE.cpp @@ -41,7 +41,7 @@ void CTE::initCTESpillContextAndPartitionConfig( = std::make_shared(operator_spill_threshold, query_id_and_cte_id, this->partitions); this->partition_config = std::make_shared( - operator_spill_threshold / this->partition_num, + operator_spill_threshold == 0 ? 0 : operator_spill_threshold / this->partition_num, spill_config, spill_block_schema, query_id_and_cte_id, @@ -88,15 +88,18 @@ CTEOpStatus CTE::tryGetBlockAt(size_t cte_reader_id, size_t partition_id, Block template CTEOpStatus CTE::pushBlock(size_t partition_id, const Block & block) { - if unlikely (block.rows() == 0) - return CTEOpStatus::OK; - std::shared_lock rw_lock(this->rw_lock); if unlikely (this->is_cancelled) return CTEOpStatus::CANCELLED; + if unlikely (block.rows() == 0) + return CTEOpStatus::OK; + + // TODO delete ------------------ this->total_recv_blocks.fetch_add(1); this->total_recv_rows.fetch_add(block.rows()); + // ------------------ + return this->partitions[partition_id]->pushBlock(block); } diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index e4e932bed7e..d741bd70fd0 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -71,10 +71,9 @@ template CTEOpStatus CTEPartition::pushBlock(const Block & block) { std::unique_lock aux_lock(*(this->aux_lock)); - this->total_blocks.fetch_add(1); + this->total_blocks.fetch_add(1); // TODO delete CTEOpStatus ret_status = CTEOpStatus::OK; - if unlikely (this->status != CTEPartitionStatus::NORMAL && block.rows() != 0) - // Block memory usage will be calculated after the finish of spill + if unlikely (this->status != CTEPartitionStatus::NORMAL) this->tmp_blocks.push_back(block); switch (this->status) @@ -91,7 +90,7 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) // Blocked in cpu pool is very bad. std::lock_guard lock(*(this->mu)); - this->memory_usage += block.bytes(); + this->memory_usage.fetch_add(block.bytes()); this->blocks.push_back(BlockWithCounter(block, static_cast(this->expected_source_num))); if constexpr (for_test) this->cv_for_test->notify_all(); @@ -99,7 +98,7 @@ CTEOpStatus CTEPartition::pushBlock(const Block & block) this->pipe_cv->notifyAll(); - if unlikely (this->exceedMemoryThresholdNoLock()) + if unlikely (this->exceedMemoryThreshold()) { this->status = CTEPartitionStatus::NEED_SPILL; ret_status = CTEOpStatus::NEED_SPILL; @@ -139,24 +138,15 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi this->first_log = false; } - // TODO remove - // auto * log = &Poco::Logger::get("LRUCache"); - String info = fmt::format("xzxdebug spill detail {} ", this->partition_id); - // Key represents logical index // Value represents physical index in `this->blocks` std::map split_idxs; auto evicted_block_num = this->getTotalEvictedBlockNumnoLock(); split_idxs.insert(std::make_pair(evicted_block_num, 0)); - info = fmt::format("{} ", info, evicted_block_num, 0); for (const auto & [cte_reader_id, logical_idx] : this->fetch_block_idxs) { - info = fmt::format("{} <{}:{}>", info, cte_reader_id, logical_idx); if (logical_idx > evicted_block_num) - { split_idxs.insert(std::make_pair(logical_idx, logical_idx - evicted_block_num)); - info = fmt::format("{} ", info, logical_idx, logical_idx - evicted_block_num); - } } auto split_iter = split_idxs.begin(); @@ -164,6 +154,7 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi auto total_block_in_memory_num = this->blocks.size(); while (split_iter != split_idxs.end()) { + // No more blocks can be spilled if (split_iter->second == this->blocks.size()) break; @@ -184,16 +175,11 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi end_iter = blocks_begin_iter + next_iter->second; } - info = fmt::format("{} ", info, spill_ranges.back().first, spill_ranges.back().second); - bool counter_is_zero = false; if (iter->counter == 0) // In one slice, all blocks' counter should be 0 or not be 0. Check it. counter_is_zero = true; - // TODO remove it - auto before_release_num = this->total_block_released_num; - while (iter != end_iter) { if (counter_is_zero) @@ -209,10 +195,6 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi ++iter; } - // TODO remove - if (before_release_num != this->total_block_released_num) - info = fmt::format("{} released_num: {}->{}", info, before_release_num, this->total_block_released_num); - if (counter_is_zero) { split_iter = next_iter; @@ -221,28 +203,25 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi RUNTIME_CHECK(!spilled_blocks.empty()); - // TODO remove - // LOG_INFO(log, "xzxdebug spill info {} total_block_in_disk_num {}, spilled_blocks.size(): {}, total_block_in_disk_num change: {}->{}", this->partition_id, this->total_block_in_disk_num, spilled_blocks.size(), this->total_block_in_disk_num, this->total_block_in_disk_num+spilled_blocks.size()); this->total_block_in_disk_num += spilled_blocks.size(); auto spiller = this->config->getSpiller(this->partition_id, this->spillers.size()); + // TODO delete ----------------- this->total_spill_blocks.fetch_add(spilled_blocks.size()); block_num.fetch_add(spilled_blocks.size()); for (auto & block : spilled_blocks) row_num.fetch_add(block.rows()); // ----------------- + spiller->spillBlocks(std::move(spilled_blocks), this->partition_id); spiller->finishSpill(); this->spillers.insert(std::make_pair(split_iter->first, std::move(spiller))); split_iter = next_iter; } - // if (this->blocks.size() > 0) - // LOG_INFO(log, "xzxdebug block change {} {} -> 0", this->partition_id, this->blocks.size()); - // LOG_INFO(log, info); this->blocks.clear(); - this->memory_usage = 0; + this->memory_usage.store(0); std::lock_guard aux_lock(*(this->aux_lock)); this->status = CTEPartitionStatus::NORMAL; @@ -266,7 +245,7 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) RUNTIME_CHECK_MSG(this->isSpillTriggeredNoLock(), "Spill should be triggered"); RUNTIME_CHECK_MSG(this->isBlockAvailableInDiskNoLock(cte_reader_id), "Requested block is not in disk"); - bool retry = false; + bool retried = false; while (true) { auto [iter, _] = this->cte_reader_restore_streams.insert(std::make_pair(cte_reader_id, nullptr)); @@ -286,11 +265,11 @@ CTEOpStatus CTEPartition::getBlockFromDisk(size_t cte_reader_id, Block & block) block = iter->second->read(); if (!block) { - RUNTIME_CHECK(!retry); + RUNTIME_CHECK(!retried); iter->second->readSuffix(); iter->second = nullptr; - retry = true; + retried = true; continue; } diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 7810af2674a..9eb7bdf310c 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -174,7 +174,7 @@ struct CTEPartition bool isSpillTriggeredNoLock() const { return this->total_block_in_disk_num > 0; } void addIdxNoLock(size_t cte_reader_id) { ++this->fetch_block_idxs[cte_reader_id]; } - bool exceedMemoryThresholdNoLock() const + bool exceedMemoryThreshold() const { // config will be nullptr in test if unlikely (this->config == nullptr) @@ -182,7 +182,7 @@ struct CTEPartition if (this->config->memory_threshold == 0) return false; - return this->memory_usage >= this->config->memory_threshold; + return this->memory_usage.load() >= this->config->memory_threshold; } template @@ -204,7 +204,7 @@ struct CTEPartition { for (const auto & block : this->tmp_blocks) { - this->memory_usage += block.bytes(); + this->memory_usage.fetch_add(block.bytes()); this->blocks.push_back(BlockWithCounter(block, static_cast(this->expected_source_num))); } tmp_blocks.clear(); @@ -236,7 +236,7 @@ struct CTEPartition std::unordered_map fetch_block_idxs; std::unique_ptr pipe_cv; - size_t memory_usage = 0; + std::atomic_size_t memory_usage = 0; const size_t expected_source_num; std::unordered_map spillers; diff --git a/dbms/src/Operators/CTEReader.cpp b/dbms/src/Operators/CTEReader.cpp index 347dab4f38b..30f4ac509c9 100644 --- a/dbms/src/Operators/CTEReader.cpp +++ b/dbms/src/Operators/CTEReader.cpp @@ -57,6 +57,7 @@ CTEOpStatus CTEReader::fetchBlockFromDisk(size_t source_id, Block & block) auto ret = this->cte->getBlockFromDisk(this->cte_reader_id, source_id, block); if (block) { + // TODO remove this->total_fetch_blocks.fetch_add(1); this->total_fetch_rows.fetch_add(block.rows()); this->total_fetch_from_disk.fetch_add(1); From ae768bd40fc2e6a660d08fbd3bd10fe3f1f715f7 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 8 Sep 2025 16:44:13 +0800 Subject: [PATCH 116/118] fix bug --- dbms/src/Operators/CTE.h | 3 ++- dbms/src/Operators/CTEPartition.cpp | 4 ++-- dbms/src/Operators/CTEPartition.h | 12 ++---------- 3 files changed, 6 insertions(+), 13 deletions(-) diff --git a/dbms/src/Operators/CTE.h b/dbms/src/Operators/CTE.h index 4ef4c61a86c..ca8439850d2 100644 --- a/dbms/src/Operators/CTE.h +++ b/dbms/src/Operators/CTE.h @@ -277,7 +277,8 @@ class CTE if unlikely (this->is_cancelled) return CTEOpStatus::CANCELLED; - if (this->partitions[partition_id]->isBlockAvailableInMemoryNoLock(cte_reader_id)) + if (this->partitions[partition_id]->isBlockAvailableInDiskNoLock(cte_reader_id) + || this->partitions[partition_id]->isBlockAvailableInMemoryNoLock(cte_reader_id)) return CTEOpStatus::OK; return this->is_eof ? CTEOpStatus::END_OF_FILE : CTEOpStatus::BLOCK_NOT_AVAILABLE; diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index d741bd70fd0..5a3c2290119 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -25,7 +25,7 @@ namespace DB size_t CTEPartition::getIdxInMemoryNoLock(size_t cte_reader_id) { auto idx = this->fetch_block_idxs[cte_reader_id]; - auto total_evicted = this->getTotalEvictedBlockNumnoLock(); + auto total_evicted = this->getTotalEvictedBlockNumNoLock(); RUNTIME_CHECK_MSG( idx >= total_evicted, "partition id: {}, idx: {}, total_evicted: {}", @@ -141,7 +141,7 @@ CTEOpStatus CTEPartition::spillBlocks(std::atomic_size_t & block_num, std::atomi // Key represents logical index // Value represents physical index in `this->blocks` std::map split_idxs; - auto evicted_block_num = this->getTotalEvictedBlockNumnoLock(); + auto evicted_block_num = this->getTotalEvictedBlockNumNoLock(); split_idxs.insert(std::make_pair(evicted_block_num, 0)); for (const auto & [cte_reader_id, logical_idx] : this->fetch_block_idxs) { diff --git a/dbms/src/Operators/CTEPartition.h b/dbms/src/Operators/CTEPartition.h index 9eb7bdf310c..902369acab3 100644 --- a/dbms/src/Operators/CTEPartition.h +++ b/dbms/src/Operators/CTEPartition.h @@ -150,7 +150,7 @@ struct CTEPartition size_t getIdxInMemoryNoLock(size_t cte_reader_id); - UInt64 getTotalEvictedBlockNumnoLock() const + UInt64 getTotalEvictedBlockNumNoLock() const { return this->total_block_released_num + this->total_block_in_disk_num; } @@ -164,7 +164,7 @@ struct CTEPartition this->partition_id, idx, this->total_block_released_num); - return idx < this->getTotalEvictedBlockNumnoLock(); + return idx < this->getTotalEvictedBlockNumNoLock(); } bool isBlockAvailableInMemoryNoLock(size_t cte_reader_id) @@ -191,14 +191,6 @@ struct CTEPartition CTEOpStatus spillBlocks(std::atomic_size_t & block_num, std::atomic_size_t & row_num); CTEOpStatus getBlockFromDisk(size_t cte_reader_id, Block & block); - bool isBlockAvailableNoLock(size_t cte_reader_id) - { - if (this->isBlockAvailableInDiskNoLock(cte_reader_id)) - return true; - - return this->isBlockAvailableInMemoryNoLock(cte_reader_id); - } - // Need aux_lock and mu void putTmpBlocksIntoBlocksNoLock() { From 53a4141ec5074c68aecc0ff28500e2f48167d7da Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 11 Sep 2025 10:52:07 +0800 Subject: [PATCH 117/118] refine memory_usage --- dbms/src/Operators/CTEPartition.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Operators/CTEPartition.cpp b/dbms/src/Operators/CTEPartition.cpp index 5a3c2290119..85b7e98cfef 100644 --- a/dbms/src/Operators/CTEPartition.cpp +++ b/dbms/src/Operators/CTEPartition.cpp @@ -51,12 +51,15 @@ CTEOpStatus CTEPartition::tryGetBlock(size_t cte_reader_id, Block & block) if (!this->isBlockAvailableInMemoryNoLock(cte_reader_id)) return CTEOpStatus::BLOCK_NOT_AVAILABLE; - auto idx = this->getIdxInMemoryNoLock(cte_reader_id); + const auto idx = this->getIdxInMemoryNoLock(cte_reader_id); block = this->blocks[idx].block; assert(this->blocks[idx].counter > 0); if ((--this->blocks[idx].counter) == 0) + { + this->memory_usage.fetch_sub(this->blocks[idx].block.bytes()); this->blocks[idx].block.clear(); + } // TODO delete ------------- { auto [iter, _] = this->fetch_in_mem_idxs.insert(std::make_pair(cte_reader_id, 0)); From f7373af2009990ed9feeb55e1789bcdc47ac9272 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 27 Oct 2025 10:00:00 +0800 Subject: [PATCH 118/118] address comments --- dbms/src/Operators/CTESinkOp.cpp | 4 ++-- dbms/src/Operators/CTESourceOp.cpp | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/src/Operators/CTESinkOp.cpp b/dbms/src/Operators/CTESinkOp.cpp index e6fd92f7fec..88174fc583c 100644 --- a/dbms/src/Operators/CTESinkOp.cpp +++ b/dbms/src/Operators/CTESinkOp.cpp @@ -38,7 +38,7 @@ OperatorStatus CTESinkOp::writeImpl(Block && block) { case CTEOpStatus::WAIT_SPILL: // CTE is spilling blocks to disk, we need to wait the finish of spill - DB::setNotifyFuture(&(this->io_notifier)); + setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::NEED_SPILL: return OperatorStatus::IO_OUT; @@ -60,7 +60,7 @@ OperatorStatus CTESinkOp::executeIOImpl() return OperatorStatus::NEED_INPUT; case CTEOpStatus::WAIT_SPILL: // CTE is spilling blocks to disk, we need to wait the finish of spill - DB::setNotifyFuture(&(this->io_notifier)); + setNotifyFuture(&(this->io_notifier)); return OperatorStatus::WAIT_FOR_NOTIFY; case CTEOpStatus::CANCELLED: return OperatorStatus::CANCELLED; diff --git a/dbms/src/Operators/CTESourceOp.cpp b/dbms/src/Operators/CTESourceOp.cpp index c2dd0bc508b..f24e91c3219 100644 --- a/dbms/src/Operators/CTESourceOp.cpp +++ b/dbms/src/Operators/CTESourceOp.cpp @@ -30,8 +30,7 @@ OperatorStatus CTESourceOp::readImpl(Block & block) { if unlikely (this->block_from_disk) { - block = this->block_from_disk; - this->block_from_disk.clear(); + block.swap(block_from_disk); return OperatorStatus::HAS_OUTPUT; }